From f52ecd617f020efbfa76267eef0e73f7c2858026 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Fri, 25 Feb 2022 10:02:24 -0500 Subject: [PATCH 1/9] kv: don't consult synthetic timestamp bit before commit-wait This commit removes "fast-paths" that skipped commit-wait when a transaction committed with a non-synthetic timestamp. In the future, once we completely decouple transaction timestamps from clock timestamps, we will stop setting the synthetic bit on transaction timestamps, even if those timestamps lead the HLC clock. This commit takes the first step in that direction. --- pkg/kv/kvclient/kvcoord/helpers_test.go | 6 ++++++ pkg/kv/kvclient/kvcoord/testing_knobs.go | 4 ++++ pkg/kv/kvclient/kvcoord/txn_coord_sender.go | 19 +++++++++++-------- .../kvclient/kvcoord/txn_coord_sender_test.go | 13 ++++++++++--- .../kvserver/batcheval/cmd_end_transaction.go | 11 +++++------ pkg/kv/kvserver/replica_send.go | 14 +++++--------- pkg/kv/kvserver/replica_tscache.go | 18 ------------------ 7 files changed, 41 insertions(+), 44 deletions(-) diff --git a/pkg/kv/kvclient/kvcoord/helpers_test.go b/pkg/kv/kvclient/kvcoord/helpers_test.go index 31df75292002..5df39946255a 100644 --- a/pkg/kv/kvclient/kvcoord/helpers_test.go +++ b/pkg/kv/kvclient/kvcoord/helpers_test.go @@ -55,3 +55,9 @@ func (tcf *TxnCoordSenderFactory) TestingSetLinearizable(linearizable bool) { func (tcf *TxnCoordSenderFactory) TestingSetMetrics(metrics TxnMetrics) { tcf.metrics = metrics } + +// TestingSetCommitWaitFilter allows tests to instrument the beginning of a +// transaction commit wait sleep. +func (tcf *TxnCoordSenderFactory) TestingSetCommitWaitFilter(filter func()) { + tcf.testingKnobs.CommitWaitFilter = filter +} diff --git a/pkg/kv/kvclient/kvcoord/testing_knobs.go b/pkg/kv/kvclient/kvcoord/testing_knobs.go index 81435ceb6a18..9674aa6df967 100644 --- a/pkg/kv/kvclient/kvcoord/testing_knobs.go +++ b/pkg/kv/kvclient/kvcoord/testing_knobs.go @@ -48,6 +48,10 @@ type ClientTestingKnobs struct { // DisableCommitSanityCheck allows "setting" the DisableCommitSanityCheck to // true without actually overriding the variable. DisableCommitSanityCheck bool + + // CommitWaitFilter allows tests to instrument the beginning of a transaction + // commit wait sleep. + CommitWaitFilter func() } var _ base.ModuleTestingKnobs = &ClientTestingKnobs{} diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go index 2e2a57eb5620..6d840203e246 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go @@ -587,7 +587,8 @@ func (tc *TxnCoordSender) Send( // clock to read below that future-time value, violating "monotonic reads". // // In practice, most transactions do not need to wait at all, because their -// commit timestamps were pulled from an HLC clock (i.e. are not synthetic) +// commit timestamps were pulled from an HLC clock (either the local clock +// or a remote clock on a node whom the local node has communicated with) // and so they will be guaranteed to lead the local HLC's clock, assuming // proper HLC time propagation. Only transactions whose commit timestamps // were pushed into the future will need to wait, like those who wrote to a @@ -640,18 +641,20 @@ func (tc *TxnCoordSender) maybeCommitWait(ctx context.Context, deferred bool) er commitTS := tc.mu.txn.WriteTimestamp readOnly := tc.mu.txn.Sequence == 0 linearizable := tc.linearizable - needWait := commitTS.Synthetic || (linearizable && !readOnly) - if !needWait { - // No need to wait. If !Synthetic then we know the commit timestamp - // leads the local HLC clock, and since that's all we'd need to wait - // for, we can short-circuit. - return nil - } waitUntil := commitTS if linearizable && !readOnly { waitUntil = waitUntil.Add(tc.clock.MaxOffset().Nanoseconds(), 0) } + if waitUntil.LessEq(tc.clock.Now()) { + // No wait fast-path. This is the common case for most transactions. Only + // transactions who have their commit timestamp bumped into the future will + // need to wait. + return nil + } + if fn := tc.testingKnobs.CommitWaitFilter; fn != nil { + fn() + } before := tc.clock.PhysicalTime() est := waitUntil.GoTime().Sub(before) diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go index aa178d762c1e..d14fa14e81d5 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go @@ -1349,8 +1349,12 @@ func TestTxnCommitWait(t *testing.T) { // testFn := func(t *testing.T, linearizable, commit, readOnly, futureTime, deferred bool) { s, metrics, cleanupFn := setupMetricsTest(t) - s.DB.GetFactory().(*kvcoord.TxnCoordSenderFactory).TestingSetLinearizable(linearizable) defer cleanupFn() + s.DB.GetFactory().(*kvcoord.TxnCoordSenderFactory).TestingSetLinearizable(linearizable) + commitWaitC := make(chan struct{}) + s.DB.GetFactory().(*kvcoord.TxnCoordSenderFactory).TestingSetCommitWaitFilter(func() { + close(commitWaitC) + }) // maxClockOffset defines the maximum clock offset between nodes in the // cluster. When in linearizable mode, all writing transactions must @@ -1361,7 +1365,7 @@ func TestTxnCommitWait(t *testing.T) { // gateway they use. This ensures that all causally dependent // transactions commit with higher timestamps, even if their read and // writes sets do not conflict with the original transaction's. This, in - // turn, prevents the "causal reverse" anamoly which can be observed by + // turn, prevents the "causal reverse" anomaly which can be observed by // a third, concurrent transaction. See the following blog post for // more: https://www.cockroachlabs.com/blog/consistency-model/. maxClockOffset := s.Clock.MaxOffset() @@ -1468,8 +1472,10 @@ func TestTxnCommitWait(t *testing.T) { // Advance the manual clock slowly. If the commit-wait sleep completes // too early, we'll catch it with the require.Empty. If it completes too - // late, we'll stall when pulling from the channel. + // late, we'll stall when pulling from the channel. Before doing so, wait + // until the transaction has begun its commit-wait. for expWait > 0 { + <-commitWaitC require.Empty(t, errC) adv := futureOffset / 5 @@ -2297,6 +2303,7 @@ func TestTxnRequestTxnTimestamp(t *testing.T) { return err } } + manual.Set(txn.ProvisionalCommitTimestamp().WallTime) return nil }); err != nil { t.Fatal(err) diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 62d919dc9305..be14b556213e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -653,12 +653,11 @@ func RunCommitTrigger( // side-effects beyond those of the intents that it has written. // The transaction should not have a commit timestamp in the future of present - // time, even it its commit timestamp is synthetic. Such cases should be - // caught in maybeCommitWaitBeforeCommitTrigger before getting here, which - // should sleep for long enough to ensure that the local clock leads the - // commit timestamp. An error here may indicate that the transaction's commit - // timestamp was bumped after it acquired latches. - if txn.WriteTimestamp.Synthetic && rec.Clock().Now().Less(txn.WriteTimestamp) { + // time. Such cases should be caught in maybeCommitWaitBeforeCommitTrigger + // before getting here, which should sleep for long enough to ensure that the + // local clock leads the commit timestamp. An error here may indicate that the + // transaction's commit timestamp was bumped after it acquired latches. + if rec.Clock().Now().Less(txn.WriteTimestamp) { return result.Result{}, errors.AssertionFailedf("txn %s with %s commit trigger needs "+ "commit wait. Was its timestamp bumped after acquiring latches?", txn, ct.Kind()) } diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 3295886e91db..ae718eb43a63 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -270,14 +270,10 @@ func (r *Replica) maybeCommitWaitBeforeCommitTrigger( // instead, we commit wait here and then assert that transactions with commit // triggers do not need to commit wait further by the time they reach command // evaluation. - // - // NOTE: just like in TxnCoordSender.maybeCommitWait, we only need to perform - // a commit-wait sleep if the commit timestamp is "synthetic". Otherwise, it - // is known not to be in advance of present time. - if !txn.WriteTimestamp.Synthetic { - return nil - } - if !r.Clock().Now().Less(txn.WriteTimestamp) { + if txn.WriteTimestamp.LessEq(r.Clock().Now()) { + // No wait fast-path. This is the common case for most transactions. Only + // transactions who have their commit timestamp bumped into the future will + // need to wait. return nil } @@ -789,7 +785,7 @@ func (r *Replica) handleReadWithinUncertaintyIntervalError( if !canDoServersideRetry(ctx, pErr, ba, nil /* br */, nil /* g */, nil /* deadline */) { return nil, pErr } - if ba.Txn == nil && ba.Timestamp.Synthetic { + if ba.Txn == nil { // If the request is non-transactional and it was refreshed into the future // after observing a value with a timestamp in the future, immediately sleep // until its new read timestamp becomes present. We don't need to do this diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index e71a8b32d8e8..d414b9a07666 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -47,24 +47,6 @@ func (r *Replica) addToTSCacheChecked( "cache after evaluating %v (resp=%v; err=%v) with lease expiration %v. The timestamp "+ "cache update could be lost of a non-cooperative lease change.", ts, ba, br, pErr, exp) } - // All updates the to timestamp cache with non-synthetic timestamps must be - // performed at or below the current time. This is no longer strictly - // required for correctness as lease transfers now read the timestamp cache - // directly instead of using the local HLC clock as a proxy for its high - // water-mark, but it serves as a good proxy for proper handling of HLC - // clock updates and, by extension, observed timestamps. - // - // TODO(nvanbenschoten): this is currently disabled because we seem to - // regularly hit it on master. Now that we ship a snapshot of the timestamp - // cache on lease transfers instead of just the current clock time, the - // property this is asserting is no longer quite as important, so we can - // disable the check. However, it would still be nice to track down how we - // can hit this. - if !ts.Synthetic && st.Now.ToTimestamp().Less(ts) && false { - log.Fatalf(ctx, "Unsafe timestamp cache update! Cannot add timestamp %s to timestamp "+ - "cache after evaluating %v (resp=%v; err=%v) with local hlc clock at timestamp %s. "+ - "Non-synthetic timestamps should always lag the local hlc clock.", ts, ba, br, pErr, st.Now) - } r.store.tsCache.Add(start, end, ts, txnID) } From 186e9a8abb4dce125e15ed1d95e0c04fd6e128b6 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 2 Mar 2022 22:01:52 -0500 Subject: [PATCH 2/9] kv: introduce and plumb intent resolution "pending observations" This commit introduces a new `ClockWhilePending` field to intent resolution requests. This field is an optional clock observation from the intent's leaseholder node that was captured at some point before the intent's transaction was pushed and found to be PENDING. The clock observation is used to forward the intent's local timestamp during intent resolution. If the clock observation was captured from a different node than the node which evaluates the ResolveIntent request, it will be ignored and the intent's local timestamp will not be changed. The field is not yet read, but is written and will be read by a later commit. --- .../kvserver/batcheval/cmd_resolve_intent.go | 4 + .../batcheval/cmd_resolve_intent_range.go | 4 + pkg/kv/kvserver/batcheval/eval_context.go | 3 +- .../concurrency/concurrency_manager.go | 1 + .../concurrency/concurrency_manager_test.go | 7 +- .../kvserver/concurrency/lock_table_waiter.go | 118 +++++++++++++++++- .../concurrency/lock_table_waiter_test.go | 15 ++- .../testdata/concurrency_manager/basic | 2 +- .../testdata/concurrency_manager/uncertainty | 6 +- .../testdata/concurrency_manager/update | 6 +- .../intentresolver/intent_resolver.go | 24 ++-- pkg/roachpb/api.go | 18 +-- pkg/roachpb/api.proto | 26 ++++ pkg/roachpb/data.proto | 6 +- 14 files changed, 199 insertions(+), 41 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go index 6cc66351e1b5..4cc884af5c60 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go @@ -89,6 +89,10 @@ func ResolveIntent( } update := args.AsLockUpdate() + if update.ClockWhilePending.NodeID != cArgs.EvalCtx.NodeID() { + // The observation was from the wrong node. Ignore. + update.ClockWhilePending = roachpb.ObservedTimestamp{} + } ok, err := storage.MVCCResolveWriteIntent(ctx, readWriter, ms, update) if err != nil { return result.Result{}, err diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go index d66eb4e2c8a6..79f2e8728040 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go @@ -48,6 +48,10 @@ func ResolveIntentRange( } update := args.AsLockUpdate() + if update.ClockWhilePending.NodeID != cArgs.EvalCtx.NodeID() { + // The observation was from the wrong node. Ignore. + update.ClockWhilePending = roachpb.ObservedTimestamp{} + } numKeys, resumeSpan, err := storage.MVCCResolveWriteIntentRange( ctx, readWriter, ms, update, h.MaxSpanRequestKeys) if err != nil { diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index ff157e6cb12a..0a3612472e8a 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -163,6 +163,7 @@ type MockEvalCtx struct { ClusterSettings *cluster.Settings Desc *roachpb.RangeDescriptor StoreID roachpb.StoreID + NodeID roachpb.NodeID Clock *hlc.Clock Stats enginepb.MVCCStats QPS float64 @@ -208,7 +209,7 @@ func (m *mockEvalCtxImpl) GetConcurrencyManager() concurrency.Manager { panic("unimplemented") } func (m *mockEvalCtxImpl) NodeID() roachpb.NodeID { - panic("unimplemented") + return m.MockEvalCtx.NodeID } func (m *mockEvalCtxImpl) GetNodeLocality() roachpb.Locality { panic("unimplemented") diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index 751b4e3bfb33..0ad59c235cdb 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -155,6 +155,7 @@ func NewManager(cfg Config) Manager { }, lt: lt, ltw: &lockTableWaiterImpl{ + nodeDesc: cfg.NodeDesc, st: cfg.Settings, clock: cfg.Clock, stopper: cfg.Stopper, diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index d024fe1d359a..b5daed7773a6 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -722,7 +722,12 @@ func (c *cluster) PushTransaction( func (c *cluster) ResolveIntent( ctx context.Context, intent roachpb.LockUpdate, _ intentresolver.ResolveOptions, ) *roachpb.Error { - log.Eventf(ctx, "resolving intent %s for txn %s with %s status", intent.Key, intent.Txn.ID.Short(), intent.Status) + var obsStr string + if obs := intent.ClockWhilePending; obs != (roachpb.ObservedTimestamp{}) { + obsStr = fmt.Sprintf(" and clock observation {%d %v}", obs.NodeID, obs.Timestamp) + } + log.Eventf(ctx, "resolving intent %s for txn %s with %s status%s", + intent.Key, intent.Txn.ID.Short(), intent.Status, obsStr) c.m.OnLockUpdated(ctx, &intent) return nil } diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter.go b/pkg/kv/kvserver/concurrency/lock_table_waiter.go index d5f1af677004..bc0d1cc8578b 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter.go @@ -97,11 +97,12 @@ var LockTableDeadlockDetectionPushDelay = settings.RegisterDurationSetting( // lockTableWaiterImpl is an implementation of lockTableWaiter. type lockTableWaiterImpl struct { - st *cluster.Settings - clock *hlc.Clock - stopper *stop.Stopper - ir IntentResolver - lt lockTable + nodeDesc *roachpb.NodeDescriptor + st *cluster.Settings + clock *hlc.Clock + stopper *stop.Stopper + ir IntentResolver + lt lockTable // When set, WriteIntentError are propagated instead of pushing // conflicting transactions. @@ -458,6 +459,7 @@ func (w *lockTableWaiterImpl) pushLockTxn( // Construct the request header and determine which form of push to use. h := w.pushHeader(req) var pushType roachpb.PushTxnType + var beforePushObs roachpb.ObservedTimestamp switch req.WaitPolicy { case lock.WaitPolicy_Block: // This wait policy signifies that the request wants to wait until the @@ -469,6 +471,24 @@ func (w *lockTableWaiterImpl) pushLockTxn( switch ws.guardAccess { case spanset.SpanReadOnly: pushType = roachpb.PUSH_TIMESTAMP + beforePushObs = roachpb.ObservedTimestamp{ + NodeID: w.nodeDesc.NodeID, + Timestamp: w.clock.NowAsClockTimestamp(), + } + // TODO(nvanbenschoten): because information about the local_timestamp + // leading the MVCC timestamp of an intent is lost, we also need to push + // the intent up to the top of the transaction's local uncertainty limit + // on this node. This logic currently lives in pushHeader, but we could + // simplify it when removing synthetic timestamps and then move it out + // here. + // + // We could also explore adding a preserve_local_timestamp flag to + // MVCCValue that would explicitly storage the local timestamp even in + // cases where it would normally be omitted. This could be set during + // intent resolution when a push observation is provided. Or we could + // not persist this, but still preserve the local timestamp when the + // adjusting the intent, accepting that the intent would then no longer + // round-trip and would lose the local timestamp if rewritten later. log.VEventf(ctx, 2, "pushing timestamp of txn %s above %s", ws.txn.ID.Short(), h.Timestamp) case spanset.SpanReadWrite: @@ -532,12 +552,98 @@ func (w *lockTableWaiterImpl) pushLockTxn( // We always poison due to limitations of the API: not poisoning equals // clearing the AbortSpan, and if our pushee transaction first got pushed // for timestamp (by us), then (by someone else) aborted and poisoned, and - // then we run the below code, we're clearing the AbortSpan illegaly. + // then we run the below code, we're clearing the AbortSpan illegally. // Furthermore, even if our pushType is not PUSH_ABORT, we may have ended up // with the responsibility to abort the intents (for example if we find the // transaction aborted). To do better here, we need per-intent information // on whether we need to poison. resolve := roachpb.MakeLockUpdate(pusheeTxn, roachpb.Span{Key: ws.key}) + if pusheeTxn.Status == roachpb.PENDING { + // The pushee was still PENDING at the time that the push observed its + // transaction record. It is safe to use the clock observation we gathered + // before initiating the push during intent resolution, as we know that this + // observation must have been made before the pushee committed (implicitly + // or explicitly) and acknowledged its client, assuming it does commit at + // some point. + // + // This observation can be used to forward the local timestamp of the intent + // when intent resolution forwards its version timestamp. This is important, + // as it prevents the pusher, who has an even earlier observed timestamp + // from this node, from considering this intent to be uncertain after the + // resolution succeeds and the pusher returns to read. + // + // For example, consider a reader with a read timestamp of 10, a global + // uncertainty limit of 25, and a local uncertainty limit (thanks to an + // observed timestamp) of 15. The reader conflicts with an intent that has a + // version timestamp and local timestamp of 8. The reader observes the local + // clock at 16 before pushing and then succeeds in pushing the intent's + // holder txn to timestamp 11 (read_timestamp + 1). If the reader were to + // resolve the intent to timestamp 11 but leave its local timestamp at 8 + // then the reader would consider the value "uncertain" upon re-evaluation. + // However, if the reader also updates the value's local timestamp to 16 + // during intent resolution then it will not consider the value to be + // "uncertain". + // + // Unfortunately, this does not quite work as written, as the MVCC key + // encoding logic normalizes (as an optimization) keys with + // local timestamp >= mvcc timestamp + // to + // local timestamp == mvcc timestamp + // To work around this, the pusher must also push the mvcc timestamp of the + // intent above its own local uncertainty limit. In the example above, this + // would mean pushing the intent's holder txn to timestamp 16 as well. The + // logic that handles this is in pushHeader. + // + // Note that it would be incorrect to update the intent's local timestamp if + // the pushee was found to be committed (implicitly or explicitly), as the + // pushee may have already acknowledged its client by the time the clock + // observation was taken and the value should be considered uncertain. Doing + // so could allow the pusher to serve a stale read. + // + // For example, if we used the observation after the push found a committed + // pushee, we would be susceptible to a stale read that looks like: + // 1. txn1 writes intent on key k @ ts 10, on node N + // 2. txn1 commits @ ts 15, acks client + // 3. txn1's async intent resolution of key k stalls + // 4. txn2 begins after txn1 with read timestamp @ 11 + // 5. txn2 collects observed timestamp @ 12 from node N + // 6. txn2 encounters intent on key k, observes clock @ ts 13, pushes, finds + // committed record, resolves intent with observation. Committed version + // now has mvcc timestamp @ 15 and local timestamp @ 13 + // 7. txn2 reads @ 11 with local uncertainty limit @ 12, fails to observe + // key k's new version. Stale read! + // + // More subtly, it would also be incorrect to update the intent's local + // timestamp using an observation captured _after_ the push completed, even + // if it had found a PENDING record. This is because this ordering makes no + // guarantee that the clock observation is captured before the pushee + // commits and acknowledges its client. This could not lead to the pusher + // serving a stale read, but it could lead to other transactions serving + // stale reads. + // + // For example, if we captured the observation after the push completed, we + // would be susceptible to a stale read that looks like: + // 1. txn1 writes intent on key k @ ts 10, on node N + // 2. txn2 (concurrent with txn1, so no risk of stale read itself) encounters + // intent on key k, pushes, finds pending record and pushes to timestamp 14 + // 3. txn1 commits @ ts 15, acks client + // 4. txn1's async intent resolution of key k stalls + // 5. txn3 begins after txn1 with read timestamp @ 11 + // 6. txn3 collects observed timestamp @ 12 from node N + // 7. txn2 observes clock @ 13 _after_ push, resolves intent (still pending) + // with observation. Intent now has mvcc timestamp @ 14 and local + // timestamp @ 13 + // 8. txn3 reads @ 11 with local uncertainty limit @ 12, fails to observe + // key k's intent so it does not resolve it to committed. Stale read! + // + // There is some inherent raciness here, because the lease may move between + // when we push and when the reader later read. In such cases, the reader's + // local uncertainty limit may exceed the intent's local timestamp during + // the subsequent read and it may need to push again. However, we expect to + // eventually succeed in reading, either after lease movement subsides or + // after the reader's read timestamp surpasses its global uncertainty limit. + resolve.ClockWhilePending = beforePushObs + } opts := intentresolver.ResolveOptions{Poison: true} return w.ir.ResolveIntent(ctx, resolve, opts) } diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index 600ab289e456..957c687ead94 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -115,11 +115,12 @@ func setupLockTableWaiterTest() ( signal: make(chan struct{}, 1), } w := &lockTableWaiterImpl{ - st: st, - clock: hlc.NewClock(manual.UnixNano, time.Nanosecond), - stopper: stop.NewStopper(), - ir: ir, - lt: &mockLockTable{}, + nodeDesc: &roachpb.NodeDescriptor{NodeID: 1}, + st: st, + clock: hlc.NewClock(manual.UnixNano, time.Nanosecond), + stopper: stop.NewStopper(), + ir: ir, + lt: &mockLockTable{}, } return w, ir, guard, manual } @@ -367,6 +368,7 @@ func testWaitPush(t *testing.T, k waitKind, makeReq func() Request, expPushTS hl require.Equal(t, keyA, intent.Key) require.Equal(t, pusheeTxn.ID, intent.Txn.ID) require.Equal(t, roachpb.ABORTED, intent.Status) + require.Zero(t, intent.ClockWhilePending) g.state = waitingState{kind: doneWaiting} g.notify() return nil @@ -550,6 +552,7 @@ func testErrorWaitPush( require.Equal(t, keyA, intent.Key) require.Equal(t, pusheeTxn.ID, intent.Txn.ID) require.Equal(t, roachpb.ABORTED, intent.Status) + require.Zero(t, intent.ClockWhilePending) g.state = waitingState{kind: doneWaiting} g.notify() return nil @@ -736,6 +739,7 @@ func testWaitPushWithTimeout(t *testing.T, k waitKind, makeReq func() Request) { require.Equal(t, keyA, intent.Key) require.Equal(t, pusheeTxn.ID, intent.Txn.ID) require.Equal(t, roachpb.ABORTED, intent.Status) + require.Zero(t, intent.ClockWhilePending) g.state = waitingState{kind: doneWaiting} g.notify() return nil @@ -856,6 +860,7 @@ func TestLockTableWaiterDeferredIntentResolverError(t *testing.T) { require.Equal(t, keyA, intents[0].Key) require.Equal(t, pusheeTxn.ID, intents[0].Txn.ID) require.Equal(t, roachpb.ABORTED, intents[0].Status) + require.Zero(t, intents[0].ClockWhilePending) return err1 } err := w.WaitOn(ctx, req, g) diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic index 5a1289e42c39..88386f5b2af6 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic @@ -226,7 +226,7 @@ debug-advance-clock ts=123 on-txn-updated txn=txn2 status=pending ts=18,1 ---- [-] update txn: increasing timestamp of txn2 -[2] sequence req5: resolving intent "k" for txn 00000002 with PENDING status +[2] sequence req5: resolving intent "k" for txn 00000002 with PENDING status and clock observation {1 246.000000000,0} [2] sequence req5: lock wait-queue event: done waiting [2] sequence req5: conflicted with 00000002-0000-0000-0000-000000000000 on "k" for 123.000s [2] sequence req5: acquiring latches diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty index f378d83e27e8..500df03eaf04 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty @@ -47,7 +47,7 @@ sequence req=req1 on-txn-updated txn=txn1 status=pending ts=15,2 ---- [-] update txn: increasing timestamp of txn1 -[3] sequence req1: resolving intent "k" for txn 00000001 with PENDING status +[3] sequence req1: resolving intent "k" for txn 00000001 with PENDING status and clock observation {1 123.000000000,1} [3] sequence req1: lock wait-queue event: done waiting [3] sequence req1: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 0.000s [3] sequence req1: acquiring latches @@ -113,7 +113,7 @@ sequence req=req1 on-txn-updated txn=txn1 status=pending ts=135,1 ---- [-] update txn: increasing timestamp of txn1 -[3] sequence req1: resolving intent "k" for txn 00000001 with PENDING status +[3] sequence req1: resolving intent "k" for txn 00000001 with PENDING status and clock observation {1 135.000000000,1} [3] sequence req1: lock wait-queue event: done waiting [3] sequence req1: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 0.000s [3] sequence req1: acquiring latches @@ -182,7 +182,7 @@ sequence req=req1 on-txn-updated txn=txn1 status=pending ts=150,2? ---- [-] update txn: increasing timestamp of txn1 -[3] sequence req1: resolving intent "k" for txn 00000001 with PENDING status +[3] sequence req1: resolving intent "k" for txn 00000001 with PENDING status and clock observation {1 135.000000000,2} [3] sequence req1: lock wait-queue event: done waiting [3] sequence req1: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 0.000s [3] sequence req1: acquiring latches diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update index 3a46cf3d2ad3..2b717ae517c6 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update @@ -70,7 +70,7 @@ local: num=0 on-txn-updated txn=txn1 status=pending ts=12,2 ---- [-] update txn: increasing timestamp of txn1 -[2] sequence req2: resolving intent "k" for txn 00000001 with PENDING status +[2] sequence req2: resolving intent "k" for txn 00000001 with PENDING status and clock observation {1 123.000000000,1} [2] sequence req2: lock wait-queue event: done waiting [2] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 0.000s [2] sequence req2: acquiring latches @@ -191,7 +191,7 @@ local: num=0 on-txn-updated txn=txn1 status=pending ts=12,2 ---- [-] update txn: increasing timestamp of txn1 -[2] sequence req2: resolving intent "k" for txn 00000001 with PENDING status +[2] sequence req2: resolving intent "k" for txn 00000001 with PENDING status and clock observation {1 123.000000000,3} [2] sequence req2: lock wait-queue event: done waiting [2] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 0.000s [2] sequence req2: acquiring latches @@ -321,7 +321,7 @@ local: num=0 on-txn-updated txn=txn1 status=pending ts=12,2 ---- [-] update txn: increasing timestamp of txn1 -[2] sequence req2: resolving intent "k" for txn 00000001 with PENDING status +[2] sequence req2: resolving intent "k" for txn 00000001 with PENDING status and clock observation {1 123.000000000,5} [2] sequence req2: lock wait-queue event: done waiting [2] sequence req2: conflicted with 00000001-0000-0000-0000-000000000000 on "k" for 0.000s [2] sequence req2: acquiring latches diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver.go b/pkg/kv/kvserver/intentresolver/intent_resolver.go index 74af2a97264f..33d88ac45cef 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver.go @@ -869,21 +869,23 @@ func (ir *IntentResolver) ResolveIntents( var batcher *requestbatcher.RequestBatcher if len(intent.EndKey) == 0 { req = &roachpb.ResolveIntentRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(intent.Span), - IntentTxn: intent.Txn, - Status: intent.Status, - Poison: opts.Poison, - IgnoredSeqNums: intent.IgnoredSeqNums, + RequestHeader: roachpb.RequestHeaderFromSpan(intent.Span), + IntentTxn: intent.Txn, + Status: intent.Status, + Poison: opts.Poison, + IgnoredSeqNums: intent.IgnoredSeqNums, + ClockWhilePending: intent.ClockWhilePending, } batcher = ir.irBatcher } else { req = &roachpb.ResolveIntentRangeRequest{ - RequestHeader: roachpb.RequestHeaderFromSpan(intent.Span), - IntentTxn: intent.Txn, - Status: intent.Status, - Poison: opts.Poison, - MinTimestamp: opts.MinTimestamp, - IgnoredSeqNums: intent.IgnoredSeqNums, + RequestHeader: roachpb.RequestHeaderFromSpan(intent.Span), + IntentTxn: intent.Txn, + Status: intent.Status, + Poison: opts.Poison, + MinTimestamp: opts.MinTimestamp, + IgnoredSeqNums: intent.IgnoredSeqNums, + ClockWhilePending: intent.ClockWhilePending, } batcher = ir.irRangeBatcher } diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 63db9786c728..2e86bc842932 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1627,10 +1627,11 @@ func (acrr *AdminChangeReplicasRequest) Changes() []ReplicationChange { // intent request. func (rir *ResolveIntentRequest) AsLockUpdate() LockUpdate { return LockUpdate{ - Span: rir.Span(), - Txn: rir.IntentTxn, - Status: rir.Status, - IgnoredSeqNums: rir.IgnoredSeqNums, + Span: rir.Span(), + Txn: rir.IntentTxn, + Status: rir.Status, + IgnoredSeqNums: rir.IgnoredSeqNums, + ClockWhilePending: rir.ClockWhilePending, } } @@ -1638,10 +1639,11 @@ func (rir *ResolveIntentRequest) AsLockUpdate() LockUpdate { // intent range request. func (rirr *ResolveIntentRangeRequest) AsLockUpdate() LockUpdate { return LockUpdate{ - Span: rirr.Span(), - Txn: rirr.IntentTxn, - Status: rirr.Status, - IgnoredSeqNums: rirr.IgnoredSeqNums, + Span: rirr.Span(), + Txn: rirr.IntentTxn, + Status: rirr.Status, + IgnoredSeqNums: rirr.IgnoredSeqNums, + ClockWhilePending: rirr.ClockWhilePending, } } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 9aaa83f888a2..af2d17d39823 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1153,6 +1153,19 @@ message ResolveIntentRequest { (gogoproto.nullable) = false, (gogoproto.customname) = "IgnoredSeqNums" ]; + // An optional clock observation from the intent's leaseholder node that was + // captured at some point before the intent's transaction was pushed and found + // to be PENDING. The clock observation is used to forward the intent's local + // timestamp during intent resolution. + // + // If the clock observation was captured from a different node than the node + // which evaluates the ResolveIntent request, it will be ignored and the + // intent's local timestamp will not be changed. If the observation was not + // ignored in these cases then intent resolution could be allowed to push the + // local timestamp of an intent above the clock on its current leaseholder. + // This could lead to stale reads if that leaseholder later served an observed + // timestamp with a clock reading below the intent's local timestamp. + ObservedTimestamp clock_while_pending = 6 [(gogoproto.nullable) = false]; } // A ResolveIntentResponse is the return value from the @@ -1184,6 +1197,19 @@ message ResolveIntentRangeRequest { (gogoproto.nullable) = false, (gogoproto.customname) = "IgnoredSeqNums" ]; + // An optional clock observation from the intent's leaseholder node that was + // captured at some point before the intent's transaction was pushed and found + // to be PENDING. The clock observation is used to forward the intent's local + // timestamp during intent resolution. + // + // If the clock observation was captured from a different node than the node + // which evaluates the ResolveIntentRange request, it will be ignored and the + // intent's local timestamp will not be changed. If the observation was not + // ignored in these cases then intent resolution could be allowed to push the + // local timestamp of an intent above the clock on its current leaseholder. + // This could lead to stale reads if that leaseholder later served an observed + // timestamp with a clock reading below the intent's local timestamp. + ObservedTimestamp clock_while_pending = 7 [(gogoproto.nullable) = false]; } // A ResolveIntentRangeResponse is the return value from the diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 886c50e29ed4..f838acca0e97 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -535,13 +535,15 @@ message LockAcquisition { // A LockUpdate is a Span together with Transaction state. LockUpdate messages // are used to update all locks held by the transaction within the span to the // transaction's authoritative state. As such, the message is used as input -// argument to intent resolution, to pass the current txn status, timestamps and -// ignored seqnum ranges to the resolution algorithm. +// argument to intent resolution, to pass the current txn status, timestamps, +// ignored seqnum ranges, and clock observations to the resolution algorithm. +// For details about these arguments, see the comments on ResolveIntentRequest. message LockUpdate { Span span = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; storage.enginepb.TxnMeta txn = 2 [(gogoproto.nullable) = false]; TransactionStatus status = 3; repeated storage.enginepb.IgnoredSeqNumRange ignored_seqnums = 4 [(gogoproto.nullable) = false, (gogoproto.customname) = "IgnoredSeqNums"]; + ObservedTimestamp clock_while_pending = 5 [(gogoproto.nullable) = false]; } // A LockStateInfo represents the metadata of a lock tracked in a replica's From ed68bfb3ac020fbae594646e5c270233c9134827 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 2 Mar 2022 22:08:11 -0500 Subject: [PATCH 3/9] kv: plumb LeaseStatus into batch evaluation Unused plumbing in preparation for a later commit. --- pkg/kv/kvserver/batcheval/declare.go | 2 ++ pkg/kv/kvserver/replica_evaluate.go | 10 ++++++++- pkg/kv/kvserver/replica_evaluate_test.go | 1 + pkg/kv/kvserver/replica_gossip.go | 4 ++-- pkg/kv/kvserver/replica_proposal.go | 9 ++++---- pkg/kv/kvserver/replica_raft.go | 4 ++-- pkg/kv/kvserver/replica_read.go | 6 +++-- pkg/kv/kvserver/replica_test.go | 28 +++++++++++++----------- pkg/kv/kvserver/replica_write.go | 23 ++++++++++++------- 9 files changed, 55 insertions(+), 32 deletions(-) diff --git a/pkg/kv/kvserver/batcheval/declare.go b/pkg/kv/kvserver/batcheval/declare.go index 101e1076ddeb..e94c23c272f7 100644 --- a/pkg/kv/kvserver/batcheval/declare.go +++ b/pkg/kv/kvserver/batcheval/declare.go @@ -20,6 +20,7 @@ import ( "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. @@ -117,6 +118,7 @@ type CommandArgs struct { EvalCtx EvalContext Header roachpb.Header Args roachpb.Request + Now hlc.ClockTimestamp // *Stats should be mutated to reflect any writes made by the command. Stats *enginepb.MVCCStats Uncertainty uncertainty.Interval diff --git a/pkg/kv/kvserver/replica_evaluate.go b/pkg/kv/kvserver/replica_evaluate.go index 4302cb05a5e6..a1c43e3acc57 100644 --- a/pkg/kv/kvserver/replica_evaluate.go +++ b/pkg/kv/kvserver/replica_evaluate.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -148,6 +149,7 @@ func evaluateBatch( rec batcheval.EvalContext, ms *enginepb.MVCCStats, ba *roachpb.BatchRequest, + st *kvserverpb.LeaseStatus, ui uncertainty.Interval, readOnly bool, ) (_ *roachpb.BatchResponse, _ result.Result, retErr *roachpb.Error) { @@ -268,7 +270,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, ui) + ctx, readWriter, rec, ms, baHeader, args, reply, st, ui) if filter := rec.EvalKnobs().TestingPostEvalFilter; filter != nil { filterArgs := kvserverbase.FilterArgs{ @@ -475,16 +477,22 @@ func evaluateCommand( h roachpb.Header, args roachpb.Request, reply roachpb.Response, + st *kvserverpb.LeaseStatus, ui uncertainty.Interval, ) (result.Result, error) { var err error var pd result.Result if cmd, ok := batcheval.LookupCommand(args.Method()); ok { + var now hlc.ClockTimestamp + if st != nil { + now = st.Now + } cArgs := batcheval.CommandArgs{ EvalCtx: rec, Header: h, Args: args, + Now: now, Stats: ms, Uncertainty: ui, } diff --git a/pkg/kv/kvserver/replica_evaluate_test.go b/pkg/kv/kvserver/replica_evaluate_test.go index 8d5c0f2e92ec..fb1ce01672f9 100644 --- a/pkg/kv/kvserver/replica_evaluate_test.go +++ b/pkg/kv/kvserver/replica_evaluate_test.go @@ -665,6 +665,7 @@ func TestEvaluateBatch(t *testing.T) { d.MockEvalCtx.EvalContext(), &d.ms, &d.ba, + nil, /* st */ uncertainty.Interval{}, d.readOnly, ) diff --git a/pkg/kv/kvserver/replica_gossip.go b/pkg/kv/kvserver/replica_gossip.go index 2ce320d8effc..a8a4da9e7757 100644 --- a/pkg/kv/kvserver/replica_gossip.go +++ b/pkg/kv/kvserver/replica_gossip.go @@ -182,7 +182,7 @@ func (r *Replica) MaybeGossipNodeLivenessRaftMuLocked( defer rw.Close() br, result, pErr := - evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, &ba, uncertainty.Interval{}, true /* readOnly */) + evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, &ba, nil /* st */, uncertainty.Interval{}, true /* readOnly */) if pErr != nil { return errors.Wrapf(pErr.GoError(), "couldn't scan node liveness records in span %s", span) } @@ -228,7 +228,7 @@ func (r *Replica) loadSystemConfig(ctx context.Context) (*config.SystemConfigEnt defer rw.Close() br, result, pErr := evaluateBatch( - ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, &ba, uncertainty.Interval{}, true, /* readOnly */ + ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, &ba, nil /* st */, uncertainty.Interval{}, true, /* readOnly */ ) if pErr != nil { return nil, pErr.GoError() diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 50f8e1a10a66..47088aea8345 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -659,6 +659,7 @@ func (r *Replica) evaluateProposal( ctx context.Context, idKey kvserverbase.CmdIDKey, ba *roachpb.BatchRequest, + st *kvserverpb.LeaseStatus, ui uncertainty.Interval, g *concurrency.Guard, ) (*result.Result, bool, *roachpb.Error) { @@ -676,7 +677,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, ui, g) + batch, ms, br, res, pErr := r.evaluateWriteBatch(ctx, idKey, ba, st, ui, g) // Note: reusing the proposer's batch when applying the command on the // proposer was explored as an optimization but resulted in no performance @@ -772,11 +773,11 @@ func (r *Replica) requestToProposal( ctx context.Context, idKey kvserverbase.CmdIDKey, ba *roachpb.BatchRequest, - st kvserverpb.LeaseStatus, + st *kvserverpb.LeaseStatus, ui uncertainty.Interval, g *concurrency.Guard, ) (*ProposalData, *roachpb.Error) { - res, needConsensus, pErr := r.evaluateProposal(ctx, idKey, ba, ui, g) + res, needConsensus, pErr := r.evaluateProposal(ctx, idKey, ba, st, ui, g) // Fill out the results even if pErr != nil; we'll return the error below. proposal := &ProposalData{ @@ -785,7 +786,7 @@ func (r *Replica) requestToProposal( doneCh: make(chan proposalResult, 1), Local: &res.Local, Request: ba, - leaseStatus: st, + leaseStatus: *st, } if needConsensus { diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 8b8a1ddbf106..8754d90acdca 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -104,7 +104,7 @@ func (r *Replica) evalAndPropose( ctx context.Context, ba *roachpb.BatchRequest, g *concurrency.Guard, - st kvserverpb.LeaseStatus, + st *kvserverpb.LeaseStatus, ui uncertainty.Interval, tok TrackedRequestToken, ) (chan proposalResult, func(), kvserverbase.CmdIDKey, *roachpb.Error) { @@ -124,7 +124,7 @@ func (r *Replica) evalAndPropose( // Attach the endCmds to the proposal and assume responsibility for // releasing the concurrency guard if the proposal makes it to Raft. - proposal.ec = endCmds{repl: r, g: g, st: st} + proposal.ec = endCmds{repl: r, g: g, st: *st} // Pull out proposal channel to return. proposal.doneCh may be set to // nil if it is signaled in this function. diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index ff64126729f4..4c899652ee70 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -87,7 +88,7 @@ func (r *Replica) executeReadOnlyBatch( // the latches are released. var result result.Result - br, result, pErr = r.executeReadOnlyBatchWithServersideRefreshes(ctx, rw, rec, ba, ui, g) + br, result, pErr = r.executeReadOnlyBatchWithServersideRefreshes(ctx, rw, rec, ba, &st, ui, g) // If the request hit a server-side concurrency retry error, immediately // propagate the error. Don't assume ownership of the concurrency guard. @@ -243,6 +244,7 @@ func (r *Replica) executeReadOnlyBatchWithServersideRefreshes( rw storage.ReadWriter, rec batcheval.EvalContext, ba *roachpb.BatchRequest, + st *kvserverpb.LeaseStatus, ui uncertainty.Interval, g *concurrency.Guard, ) (br *roachpb.BatchResponse, res result.Result, pErr *roachpb.Error) { @@ -294,7 +296,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, ui, true /* readOnly */) + br, res, pErr = evaluateBatch(ctx, kvserverbase.CmdIDKey(""), rw, rec, nil, ba, st, 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, g, nil /* deadline */) { diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 47a2a6d067d8..a30aac136395 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -530,7 +530,7 @@ func sendLeaseRequest(r *Replica, l *roachpb.Lease) error { } ba.Add(leaseReq) _, tok := r.mu.proposalBuf.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) - ch, _, _, pErr := r.evalAndPropose(ctx, &ba, allSpansGuard(), st, uncertainty.Interval{}, 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. @@ -1391,7 +1391,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, uncertainty.Interval{}, 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. @@ -7952,7 +7952,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, uncertainty.Interval{}, tok.Move(ctx)) + ch, _, id, err := repl.evalAndPropose(ctx, &ba, allSpansGuard(), &st, uncertainty.Interval{}, tok.Move(ctx)) if err != nil { t.Fatal(err) } @@ -8023,7 +8023,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, uncertainty.Interval{}, tok.Move(ctx)) + ch, _, _, err := tc.repl.evalAndPropose(ctx, &ba, allSpansGuard(), &st, uncertainty.Interval{}, tok.Move(ctx)) if err != nil { t.Fatal(err) } @@ -8137,7 +8137,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, uncertainty.Interval{}, allSpansGuard()) + cmd, pErr := r.requestToProposal(ctx, kvserverbase.CmdIDKey(id), &ba, &st, uncertainty.Interval{}, allSpansGuard()) if pErr != nil { t.Fatal(pErr) } @@ -8259,7 +8259,8 @@ func TestReplicaRefreshMultiple(t *testing.T) { incCmdID = makeIDKey() atomic.StoreInt32(&filterActive, 1) - proposal, pErr := repl.requestToProposal(ctx, incCmdID, &ba, repl.CurrentLeaseStatus(ctx), uncertainty.Interval{}, allSpansGuard()) + st := repl.CurrentLeaseStatus(ctx) + proposal, pErr := repl.requestToProposal(ctx, incCmdID, &ba, &st, uncertainty.Interval{}, allSpansGuard()) if pErr != nil { t.Fatal(pErr) } @@ -8964,7 +8965,7 @@ func TestReplicaEvaluationNotTxnMutation(t *testing.T) { assignSeqNumsForReqs(txn, &txnPut, &txnPut2) origTxn := txn.Clone() - batch, _, _, _, pErr := tc.repl.evaluateWriteBatch(ctx, makeIDKey(), &ba, uncertainty.Interval{}, allSpansGuard()) + batch, _, _, _, pErr := tc.repl.evaluateWriteBatch(ctx, makeIDKey(), &ba, nil, uncertainty.Interval{}, allSpansGuard()) defer batch.Close() if pErr != nil { t.Fatal(pErr) @@ -9687,7 +9688,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, uncertainty.Interval{}, tok.Move(ctx)) + ch, _, _, pErr := repl.evalAndPropose(ctx, &ba, allSpansGuard(), &st, uncertainty.Interval{}, tok.Move(ctx)) if pErr != nil { t.Fatal(pErr) } @@ -9735,7 +9736,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, uncertainty.Interval{}, tok.Move(ctx)) + ch, _, _, pErr := repl.evalAndPropose(ctx, &ba, allSpansGuard(), &st, uncertainty.Interval{}, tok.Move(ctx)) if pErr != nil { t.Fatal(pErr) } @@ -9800,7 +9801,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, uncertainty.Interval{}, tok.Move(ctx)) + _, _, _, pErr := repl.evalAndPropose(ctx, &ba, allSpansGuard(), &st, uncertainty.Interval{}, tok.Move(ctx)) if pErr != nil { t.Fatal(pErr) } @@ -9819,7 +9820,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, uncertainty.Interval{}, tok.Move(ctx)) + ch, _, _, pErr = repl.evalAndPropose(ctx, &ba2, allSpansGuard(), &st, uncertainty.Interval{}, tok.Move(ctx)) if pErr != nil { t.Fatal(pErr) } @@ -13272,7 +13273,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, uncertainty.Interval{}, tok) + ch, _, _, pErr := tc.repl.evalAndPropose(tracedCtx, &ba, allSpansGuard(), &st, uncertainty.Interval{}, tok) if pErr != nil { t.Fatal(pErr) } @@ -13441,7 +13442,8 @@ 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), uncertainty.Interval{}, allSpansGuard()) + st := tc.repl.CurrentLeaseStatus(ctx) + proposal, err := tc.repl.requestToProposal(ctx, cmdIDKey, &ba, &st, uncertainty.Interval{}, allSpansGuard()) if err != nil { t.Error(err) } diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index c12fe8375f53..a1bd5f41d0f7 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -164,7 +164,7 @@ func (r *Replica) executeWriteBatch( // If the command is proposed to Raft, ownership of and responsibility for // the concurrency guard will be assumed by Raft, so provide the guard to // evalAndPropose. - ch, abandon, _, pErr := r.evalAndPropose(ctx, ba, g, st, ui, tok.Move(ctx)) + ch, abandon, _, pErr := r.evalAndPropose(ctx, ba, g, &st, ui, tok.Move(ctx)) if pErr != nil { if cErr, ok := pErr.GetDetail().(*roachpb.ReplicaCorruptionError); ok { // Need to unlock here because setCorruptRaftMuLock needs readOnlyCmdMu not held. @@ -382,6 +382,7 @@ func (r *Replica) evaluateWriteBatch( ctx context.Context, idKey kvserverbase.CmdIDKey, ba *roachpb.BatchRequest, + st *kvserverpb.LeaseStatus, ui uncertainty.Interval, g *concurrency.Guard, ) (storage.Batch, enginepb.MVCCStats, *roachpb.BatchResponse, result.Result, *roachpb.Error) { @@ -395,7 +396,7 @@ func (r *Replica) evaluateWriteBatch( // Attempt 1PC execution, if applicable. If not transactional or there are // indications that the batch's txn will require retry, execute as normal. if r.canAttempt1PCEvaluation(ctx, ba, g) { - res := r.evaluate1PC(ctx, idKey, ba, g) + res := r.evaluate1PC(ctx, idKey, ba, st, g) switch res.success { case onePCSucceeded: return res.batch, res.stats, res.br, res.res, nil @@ -427,7 +428,7 @@ func (r *Replica) evaluateWriteBatch( rec := NewReplicaEvalContext(ctx, r, g.LatchSpans(), ba.RequiresClosedTSOlderThanStorageSnapshot()) defer rec.Release() batch, br, res, pErr := r.evaluateWriteBatchWithServersideRefreshes( - ctx, idKey, rec, ms, ba, ui, g, nil /* deadline */) + ctx, idKey, rec, ms, ba, st, ui, g, nil /* deadline */) return batch, *ms, br, res, pErr } @@ -467,7 +468,11 @@ type onePCResult struct { // efficient - we're avoiding writing the transaction record and writing and the // immediately deleting intents. func (r *Replica) evaluate1PC( - ctx context.Context, idKey kvserverbase.CmdIDKey, ba *roachpb.BatchRequest, g *concurrency.Guard, + ctx context.Context, + idKey kvserverbase.CmdIDKey, + ba *roachpb.BatchRequest, + st *kvserverpb.LeaseStatus, + g *concurrency.Guard, ) (onePCRes onePCResult) { log.VEventf(ctx, 2, "attempting 1PC execution") @@ -503,10 +508,10 @@ func (r *Replica) evaluate1PC( ms := new(enginepb.MVCCStats) if ba.CanForwardReadTimestamp { batch, br, res, pErr = r.evaluateWriteBatchWithServersideRefreshes( - ctx, idKey, rec, ms, &strippedBa, ui, g, etArg.Deadline) + ctx, idKey, rec, ms, &strippedBa, st, ui, g, etArg.Deadline) } else { batch, br, res, pErr = r.evaluateWriteBatchWrapper( - ctx, idKey, rec, ms, &strippedBa, ui, g) + ctx, idKey, rec, ms, &strippedBa, st, ui, g) } if pErr != nil || (!ba.CanForwardReadTimestamp && ba.Timestamp != br.Timestamp) { @@ -596,6 +601,7 @@ func (r *Replica) evaluateWriteBatchWithServersideRefreshes( rec batcheval.EvalContext, ms *enginepb.MVCCStats, ba *roachpb.BatchRequest, + st *kvserverpb.LeaseStatus, ui uncertainty.Interval, g *concurrency.Guard, deadline *hlc.Timestamp, @@ -611,7 +617,7 @@ func (r *Replica) evaluateWriteBatchWithServersideRefreshes( batch.Close() } - batch, br, res, pErr = r.evaluateWriteBatchWrapper(ctx, idKey, rec, ms, ba, ui, g) + batch, br, res, pErr = r.evaluateWriteBatchWrapper(ctx, idKey, rec, ms, ba, st, ui, g) var success bool if pErr == nil { @@ -639,11 +645,12 @@ func (r *Replica) evaluateWriteBatchWrapper( rec batcheval.EvalContext, ms *enginepb.MVCCStats, ba *roachpb.BatchRequest, + st *kvserverpb.LeaseStatus, ui uncertainty.Interval, g *concurrency.Guard, ) (storage.Batch, *roachpb.BatchResponse, result.Result, *roachpb.Error) { batch, opLogger := r.newBatchedEngine(ba, g) - br, res, pErr := evaluateBatch(ctx, idKey, batch, rec, ms, ba, ui, false /* readOnly */) + br, res, pErr := evaluateBatch(ctx, idKey, batch, rec, ms, ba, st, ui, false /* readOnly */) if pErr == nil { if opLogger != nil { res.LogicalOpLog = &kvserverpb.LogicalOpLog{ From d5b3fe7c91e1da9745b337e66ad26c776fde0aee Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Wed, 2 Mar 2022 22:45:33 -0500 Subject: [PATCH 4/9] kv: plumb local timestamp into MVCC Unused plumbing in preparation for a later commit. --- pkg/ccl/storageccl/engineccl/bench_test.go | 2 +- .../storageccl/engineccl/encrypted_fs_test.go | 1 + pkg/cli/debug.go | 7 +- .../kvcoord/dist_sender_server_test.go | 4 +- pkg/kv/kvserver/abortspan/abortspan.go | 6 +- pkg/kv/kvserver/batch_spanset_test.go | 1 + .../batcheval/cmd_add_sstable_test.go | 2 +- .../batcheval/cmd_clear_range_test.go | 2 +- .../kvserver/batcheval/cmd_conditional_put.go | 6 +- pkg/kv/kvserver/batcheval/cmd_delete.go | 2 +- pkg/kv/kvserver/batcheval/cmd_delete_range.go | 4 +- .../kvserver/batcheval/cmd_end_transaction.go | 8 +- .../batcheval/cmd_end_transaction_test.go | 8 +- pkg/kv/kvserver/batcheval/cmd_export_test.go | 4 +- .../kvserver/batcheval/cmd_heartbeat_txn.go | 2 +- pkg/kv/kvserver/batcheval/cmd_increment.go | 3 +- pkg/kv/kvserver/batcheval/cmd_init_put.go | 6 +- pkg/kv/kvserver/batcheval/cmd_push_txn.go | 2 +- pkg/kv/kvserver/batcheval/cmd_put.go | 4 +- .../cmd_query_resolved_timestamp_test.go | 6 +- pkg/kv/kvserver/batcheval/cmd_recover_txn.go | 2 +- .../batcheval/cmd_recover_txn_test.go | 4 +- .../batcheval/cmd_refresh_range_bench_test.go | 2 +- .../batcheval/cmd_refresh_range_test.go | 10 +- pkg/kv/kvserver/batcheval/cmd_refresh_test.go | 4 +- .../batcheval/cmd_resolve_intent_test.go | 4 +- .../batcheval/cmd_revert_range_test.go | 10 +- pkg/kv/kvserver/batcheval/cmd_scan_test.go | 4 +- .../batcheval/cmd_truncate_log_test.go | 2 +- pkg/kv/kvserver/batcheval/intent_test.go | 4 +- pkg/kv/kvserver/batcheval/transaction_test.go | 2 +- pkg/kv/kvserver/client_merge_test.go | 4 +- pkg/kv/kvserver/consistency_queue_test.go | 2 +- pkg/kv/kvserver/gc/data_distribution_test.go | 2 +- pkg/kv/kvserver/gc/gc_test.go | 4 +- pkg/kv/kvserver/loqrecovery/apply.go | 5 +- .../kvserver/loqrecovery/recovery_env_test.go | 9 +- pkg/kv/kvserver/mvcc_gc_queue_test.go | 8 +- .../rangefeed/catchup_scan_bench_test.go | 2 +- .../kvserver/rangefeed/catchup_scan_test.go | 4 +- pkg/kv/kvserver/rangefeed/task_test.go | 2 +- .../kvserver/rditer/replica_data_iter_test.go | 2 +- pkg/kv/kvserver/readsummary/persist.go | 2 +- pkg/kv/kvserver/replica.go | 3 +- pkg/kv/kvserver/replica_destroy.go | 2 +- pkg/kv/kvserver/replica_evaluate_test.go | 2 +- pkg/kv/kvserver/replica_raft.go | 10 +- pkg/kv/kvserver/replica_raftstorage.go | 6 +- pkg/kv/kvserver/replica_test.go | 16 +- pkg/kv/kvserver/stateloader/stateloader.go | 22 +- pkg/kv/kvserver/store.go | 13 +- pkg/kv/kvserver/store_init.go | 14 +- pkg/kv/kvserver/store_snapshot.go | 2 +- pkg/kv/kvserver/store_test.go | 3 +- pkg/kv/kvserver/stores.go | 2 +- pkg/kv/kvserver/txn_wait_queue_test.go | 2 +- pkg/server/node_test.go | 2 +- pkg/server/node_tombstone_storage.go | 2 +- pkg/server/server_test.go | 5 +- pkg/server/settings_cache.go | 2 +- pkg/storage/bench_pebble_test.go | 4 +- pkg/storage/bench_test.go | 29 +- pkg/storage/engine_test.go | 8 +- pkg/storage/metamorphic/operations.go | 12 +- pkg/storage/mvcc.go | 109 +++-- pkg/storage/mvcc_history_test.go | 48 +- pkg/storage/mvcc_incremental_iterator_test.go | 55 +-- pkg/storage/mvcc_logical_ops_test.go | 16 +- pkg/storage/mvcc_stats_test.go | 58 +-- pkg/storage/mvcc_test.go | 449 +++++++++--------- pkg/storage/pebble_mvcc_scanner_test.go | 2 +- pkg/storage/pebble_test.go | 2 +- pkg/storage/sst_test.go | 2 +- 73 files changed, 563 insertions(+), 513 deletions(-) diff --git a/pkg/ccl/storageccl/engineccl/bench_test.go b/pkg/ccl/storageccl/engineccl/bench_test.go index af76e0f51033..8d50740c1083 100644 --- a/pkg/ccl/storageccl/engineccl/bench_test.go +++ b/pkg/ccl/storageccl/engineccl/bench_test.go @@ -100,7 +100,7 @@ func loadTestData( timestamp := hlc.Timestamp{WallTime: minWallTime + rand.Int63n(int64(batchTimeSpan))} value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, valueBytes)) value.InitChecksum(key) - if err := storage.MVCCPut(ctx, batch, nil, key, timestamp, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, key, timestamp, hlc.ClockTimestamp{}, value, nil); err != nil { return nil, err } } diff --git a/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go b/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go index cbd36a07ec47..e23922101241 100644 --- a/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go +++ b/pkg/ccl/storageccl/engineccl/encrypted_fs_test.go @@ -399,6 +399,7 @@ func TestPebbleEncryption2(t *testing.T) { nil, /* ms */ roachpb.Key(key), hlc.Timestamp{}, + hlc.ClockTimestamp{}, roachpb.MakeValueFromBytes([]byte(val)), nil, /* txn */ ) diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index e7d27cb0aec9..03b4b6d187a9 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -1280,7 +1280,7 @@ func removeDeadReplicas( if err != nil { return nil, errors.Wrap(err, "loading MVCCStats") } - err = storage.MVCCPutProto(ctx, batch, &ms, key, clock.Now(), nil /* txn */, &desc) + err = storage.MVCCPutProto(ctx, batch, &ms, key, clock.Now(), hlc.ClockTimestamp{}, nil, &desc) if wiErr := (*roachpb.WriteIntentError)(nil); errors.As(err, &wiErr) { if len(wiErr.Intents) != 1 { return nil, errors.Errorf("expected 1 intent, found %d: %s", len(wiErr.Intents), wiErr) @@ -1328,7 +1328,7 @@ func removeDeadReplicas( // A crude form of the intent resolution process: abort the // transaction by deleting its record. txnKey := keys.TransactionKey(intent.Txn.Key, intent.Txn.ID) - if err := storage.MVCCDelete(ctx, batch, &ms, txnKey, hlc.Timestamp{}, nil); err != nil { + if err := storage.MVCCDelete(ctx, batch, &ms, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { return nil, err } update := roachpb.LockUpdate{ @@ -1340,8 +1340,7 @@ func removeDeadReplicas( return nil, err } // With the intent resolved, we can try again. - if err := storage.MVCCPutProto(ctx, batch, &ms, key, clock.Now(), - nil /* txn */, &desc); err != nil { + if err := storage.MVCCPutProto(ctx, batch, &ms, key, clock.Now(), hlc.ClockTimestamp{}, nil, &desc); err != nil { return nil, err } } else if err != nil { diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go index 9a8b8aa3dd82..903b30ad4bcf 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go @@ -75,9 +75,7 @@ func TestRangeLookupWithOpenTransaction(t *testing.T) { key := testutils.MakeKey(keys.Meta1Prefix, roachpb.KeyMax) now := s.Clock().Now() txn := roachpb.MakeTransaction("txn", roachpb.Key("foobar"), 0, now, 0, int32(s.SQLInstanceID())) - if err := storage.MVCCPutProto( - context.Background(), s.Engines()[0], - nil, key, now, &txn, &roachpb.RangeDescriptor{}); err != nil { + if err := storage.MVCCPutProto(context.Background(), s.Engines()[0], nil, key, now, hlc.ClockTimestamp{}, &txn, &roachpb.RangeDescriptor{}); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/abortspan/abortspan.go b/pkg/kv/kvserver/abortspan/abortspan.go index 8ecc24c9ef4f..5c5bb5f4153a 100644 --- a/pkg/kv/kvserver/abortspan/abortspan.go +++ b/pkg/kv/kvserver/abortspan/abortspan.go @@ -123,7 +123,7 @@ func (sc *AbortSpan) Del( ctx context.Context, reader storage.ReadWriter, ms *enginepb.MVCCStats, txnID uuid.UUID, ) error { key := keys.AbortSpanKey(sc.rangeID, txnID) - return storage.MVCCDelete(ctx, reader, ms, key, hlc.Timestamp{}, nil /* txn */) + return storage.MVCCDelete(ctx, reader, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */) } // Put writes an entry for the specified transaction ID. @@ -136,7 +136,7 @@ func (sc *AbortSpan) Put( ) error { log.VEventf(ctx, 2, "writing abort span entry for %s", txnID.Short()) key := keys.AbortSpanKey(sc.rangeID, txnID) - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */, entry) + return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, entry) } // CopyTo copies the abort span entries to the abort span for the range @@ -181,7 +181,7 @@ func (sc *AbortSpan) CopyTo( } return storage.MVCCPutProto(ctx, w, ms, keys.AbortSpanKey(newRangeID, txnID), - hlc.Timestamp{}, nil, &entry, + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &entry, ) }); err != nil { return errors.Wrap(err, "AbortSpan.CopyTo") diff --git a/pkg/kv/kvserver/batch_spanset_test.go b/pkg/kv/kvserver/batch_spanset_test.go index d7d9a4e6ea3c..67d0718e7509 100644 --- a/pkg/kv/kvserver/batch_spanset_test.go +++ b/pkg/kv/kvserver/batch_spanset_test.go @@ -581,6 +581,7 @@ func TestSpanSetMVCCResolveWriteIntentRange(t *testing.T) { nil, // ms roachpb.Key("b"), hlc.Timestamp{WallTime: 10}, // irrelevant + hlc.ClockTimestamp{}, // irrelevant value, nil, // txn ); err != nil { diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 352c7b1807bd..7635e89a539e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -683,7 +683,7 @@ func TestEvalAddSSTable(t *testing.T) { if kv.WallTimestamp == intentTS { txn = &intentTxn } - require.NoError(t, storage.MVCCPut(ctx, b, nil, kv.Key(), kv.Timestamp(), kv.Value(), txn)) + require.NoError(t, storage.MVCCPut(ctx, b, nil, kv.Key(), kv.Timestamp(), hlc.ClockTimestamp{}, kv.Value(), txn)) } require.NoError(t, b.Commit(false)) stats := engineStats(t, engine, 0) diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go index c31a23aa22ed..84ed9a861541 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go @@ -105,7 +105,7 @@ func TestCmdClearRangeBytesThreshold(t *testing.T) { var stats enginepb.MVCCStats for i := 0; i < test.keyCount; i++ { key := roachpb.Key(fmt.Sprintf("%04d", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, hlc.Timestamp{WallTime: int64(i % 2)}, value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, &stats, key, hlc.Timestamp{WallTime: int64(i % 2)}, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_conditional_put.go b/pkg/kv/kvserver/batcheval/cmd_conditional_put.go index 17416eb641fc..4ce3ce46e2df 100644 --- a/pkg/kv/kvserver/batcheval/cmd_conditional_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_conditional_put.go @@ -67,9 +67,11 @@ func ConditionalPut( handleMissing := storage.CPutMissingBehavior(args.AllowIfDoesNotExist) var err error if args.Blind { - err = storage.MVCCBlindConditionalPut(ctx, readWriter, cArgs.Stats, args.Key, ts, args.Value, expVal, handleMissing, h.Txn) + err = storage.MVCCBlindConditionalPut( + ctx, readWriter, cArgs.Stats, args.Key, ts, cArgs.Now, args.Value, expVal, handleMissing, h.Txn) } else { - err = storage.MVCCConditionalPut(ctx, readWriter, cArgs.Stats, args.Key, ts, args.Value, expVal, handleMissing, h.Txn) + err = storage.MVCCConditionalPut( + ctx, readWriter, cArgs.Stats, args.Key, ts, cArgs.Now, args.Value, expVal, handleMissing, h.Txn) } // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld diff --git a/pkg/kv/kvserver/batcheval/cmd_delete.go b/pkg/kv/kvserver/batcheval/cmd_delete.go index 38a40b19d18c..ee293240fb25 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete.go @@ -29,7 +29,7 @@ func Delete( args := cArgs.Args.(*roachpb.DeleteRequest) h := cArgs.Header - err := storage.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, h.Txn) + err := storage.MVCCDelete(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn) // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld // without re-evaluating the batch. This behavior isn't particularly diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 60554d27ad4b..58a11e462816 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -58,8 +58,8 @@ func DeleteRange( // can update the Result's AcquiredLocks field. returnKeys := args.ReturnKeys || h.Txn != nil deleted, resumeSpan, num, err := storage.MVCCDeleteRange( - ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, h.MaxSpanRequestKeys, timestamp, h.Txn, returnKeys, - ) + ctx, readWriter, cArgs.Stats, args.Key, args.EndKey, + h.MaxSpanRequestKeys, timestamp, cArgs.Now, h.Txn, returnKeys) if err == nil && args.ReturnKeys { reply.Keys = deleted } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index be14b556213e..734c8b3065d3 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -600,7 +600,7 @@ func updateStagingTxn( txn.LockSpans = args.LockSpans txn.InFlightWrites = args.InFlightWrites txnRecord := txn.AsRecord() - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */, &txnRecord) + return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord) } // updateFinalizedTxn persists the COMMITTED or ABORTED transaction record with @@ -627,12 +627,12 @@ func updateFinalizedTxn( // BatchRequest writes. return nil } - return storage.MVCCDelete(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */) + return storage.MVCCDelete(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) } txn.LockSpans = externalLocks txn.InFlightWrites = nil txnRecord := txn.AsRecord() - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */, &txnRecord) + return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord) } // RunCommitTrigger runs the commit trigger from an end transaction request. @@ -1011,7 +1011,7 @@ func splitTriggerHelper( if err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to fetch last replica GC timestamp") } - if err := storage.MVCCPutProto(ctx, batch, nil, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{}, nil, &replicaGCTS); err != nil { + if err := storage.MVCCPutProto(ctx, batch, nil, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &replicaGCTS); err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to copy last replica GC timestamp") } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go index 19cb34232cda..3449ae7ffdf2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction_test.go @@ -908,7 +908,7 @@ func TestEndTxnUpdatesTransactionRecord(t *testing.T) { // Write the existing transaction record, if necessary. txnKey := keys.TransactionKey(txn.Key, txn.ID) if c.existingTxn != nil { - if err := storage.MVCCPutProto(ctx, batch, nil, txnKey, hlc.Timestamp{}, nil, c.existingTxn); err != nil { + if err := storage.MVCCPutProto(ctx, batch, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, c.existingTxn); err != nil { t.Fatal(err) } } @@ -1012,13 +1012,13 @@ func TestPartialRollbackOnEndTransaction(t *testing.T) { // Write a first value at key. v.SetString("a") txn.Sequence = 1 - if err := storage.MVCCPut(ctx, batch, nil, k, ts, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { t.Fatal(err) } // Write another value. v.SetString("b") txn.Sequence = 2 - if err := storage.MVCCPut(ctx, batch, nil, k, ts, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { t.Fatal(err) } @@ -1031,7 +1031,7 @@ func TestPartialRollbackOnEndTransaction(t *testing.T) { txnKey := keys.TransactionKey(txn.Key, txn.ID) if storeTxnBeforeEndTxn { txnRec := txn.AsRecord() - if err := storage.MVCCPutProto(ctx, batch, nil, txnKey, hlc.Timestamp{}, nil, &txnRec); err != nil { + if err := storage.MVCCPutProto(ctx, batch, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRec); err != nil { t.Fatal(err) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_export_test.go b/pkg/kv/kvserver/batcheval/cmd_export_test.go index d77126d00a35..837ff893d3bd 100644 --- a/pkg/kv/kvserver/batcheval/cmd_export_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_export_test.go @@ -737,7 +737,7 @@ func TestRandomKeyAndTimestampExport(t *testing.T) { valueSize := randutil.RandIntInRange(rnd, averageValueSize-100, averageValueSize+100) value := roachpb.MakeValueFromBytes(randutil.RandBytes(rnd, valueSize)) value.InitChecksum(key) - if err := storage.MVCCPut(ctx, batch, nil, key, ts, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } @@ -748,7 +748,7 @@ func TestRandomKeyAndTimestampExport(t *testing.T) { ts = hlc.Timestamp{WallTime: int64(curWallTime), Logical: int32(curLogical)} value = roachpb.MakeValueFromBytes(randutil.RandBytes(rnd, 200)) value.InitChecksum(key) - if err := storage.MVCCPut(ctx, batch, nil, key, ts, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go index ba4817001a8a..52032f7fdfa6 100644 --- a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go @@ -79,7 +79,7 @@ func HeartbeatTxn( // is up for debate. txn.LastHeartbeat.Forward(args.Now) txnRecord := txn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { return result.Result{}, err } } diff --git a/pkg/kv/kvserver/batcheval/cmd_increment.go b/pkg/kv/kvserver/batcheval/cmd_increment.go index 7e2ed30eeb7b..aac9e412ef09 100644 --- a/pkg/kv/kvserver/batcheval/cmd_increment.go +++ b/pkg/kv/kvserver/batcheval/cmd_increment.go @@ -32,7 +32,8 @@ func Increment( h := cArgs.Header reply := resp.(*roachpb.IncrementResponse) - newVal, err := storage.MVCCIncrement(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, h.Txn, args.Increment) + newVal, err := storage.MVCCIncrement( + ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, h.Txn, args.Increment) reply.NewValue = newVal // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld diff --git a/pkg/kv/kvserver/batcheval/cmd_init_put.go b/pkg/kv/kvserver/batcheval/cmd_init_put.go index 745f4e1d5c3f..1ec82e748183 100644 --- a/pkg/kv/kvserver/batcheval/cmd_init_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_init_put.go @@ -34,9 +34,11 @@ func InitPut( var err error if args.Blind { - err = storage.MVCCBlindInitPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.FailOnTombstones, h.Txn) + err = storage.MVCCBlindInitPut( + ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, args.Value, args.FailOnTombstones, h.Txn) } else { - err = storage.MVCCInitPut(ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, args.Value, args.FailOnTombstones, h.Txn) + err = storage.MVCCInitPut( + ctx, readWriter, cArgs.Stats, args.Key, h.Timestamp, cArgs.Now, args.Value, args.FailOnTombstones, h.Txn) } // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld diff --git a/pkg/kv/kvserver/batcheval/cmd_push_txn.go b/pkg/kv/kvserver/batcheval/cmd_push_txn.go index ab7c60ddf61d..8134aa90d086 100644 --- a/pkg/kv/kvserver/batcheval/cmd_push_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_push_txn.go @@ -325,7 +325,7 @@ func PushTxn( // in the timestamp cache. if ok { txnRecord := reply.PusheeTxn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { return result.Result{}, err } } diff --git a/pkg/kv/kvserver/batcheval/cmd_put.go b/pkg/kv/kvserver/batcheval/cmd_put.go index 65f260f961c1..283164a2d5ea 100644 --- a/pkg/kv/kvserver/batcheval/cmd_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_put.go @@ -54,9 +54,9 @@ func Put( } var err error if args.Blind { - err = storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, args.Value, h.Txn) + err = storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, cArgs.Now, args.Value, h.Txn) } else { - err = storage.MVCCPut(ctx, readWriter, ms, args.Key, ts, args.Value, h.Txn) + err = storage.MVCCPut(ctx, readWriter, ms, args.Key, ts, cArgs.Now, args.Value, h.Txn) } // NB: even if MVCC returns an error, it may still have written an intent // into the batch. This allows callers to consume errors like WriteTooOld diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go index 04a56500cf89..763e0326b9e0 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go @@ -42,14 +42,14 @@ func TestQueryResolvedTimestamp(t *testing.T) { return hlc.Timestamp{WallTime: ts} } writeValue := func(k string, ts int64) { - require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), nil)) + require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), hlc.ClockTimestamp{}, nil)) } writeIntent := func(k string, ts int64) { txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, makeTS(ts), 0, 1) - require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), &txn)) + require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), makeTS(ts), hlc.ClockTimestamp{}, &txn)) } writeInline := func(k string) { - require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), hlc.Timestamp{}, nil)) + require.NoError(t, storage.MVCCDelete(ctx, db, nil, roachpb.Key(k), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil)) } // Setup: (with separated intents the actual key layout in the store is not what is listed below.) diff --git a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go index dd7111d47918..15b980715090 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go @@ -216,7 +216,7 @@ func RecoverTxn( reply.RecoveredTxn.Status = roachpb.ABORTED } txnRecord := reply.RecoveredTxn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go b/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go index 97964b5b0729..e955b83a83fb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn_test.go @@ -48,7 +48,7 @@ func TestRecoverTxn(t *testing.T) { // Write the transaction record. txnKey := keys.TransactionKey(txn.Key, txn.ID) txnRecord := txn.AsRecord() - if err := storage.MVCCPutProto(ctx, db, nil, txnKey, hlc.Timestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, db, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { t.Fatal(err) } @@ -224,7 +224,7 @@ func TestRecoverTxnRecordChanged(t *testing.T) { // request is evaluated. txnKey := keys.TransactionKey(txn.Key, txn.ID) txnRecord := c.changedTxn.AsRecord() - if err := storage.MVCCPutProto(ctx, db, nil, txnKey, hlc.Timestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, db, nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &txnRecord); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go index 43c0ca0f3984..c40cebf6a81c 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go @@ -258,7 +258,7 @@ func setupData( value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, opts.valueBytes)) value.InitChecksum(key) ts := hlc.Timestamp{WallTime: int64((pos + 1) * 5)} - if err := storage.MVCCPut(ctx, batch, nil /* ms */, key, ts, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, nil /* ms */, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatal(err) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go index f65d544f5317..cd5f1523b6e8 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range_test.go @@ -70,10 +70,10 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { }, ReadTimestamp: ts1, } - if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, v, txn); err != nil { + if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, hlc.ClockTimestamp{}, v, txn); err != nil { t.Fatal(err) } - if err := storage.MVCCPut(ctx, db, nil, roachpb.Key("unused1"), ts4, v, nil); err != nil { + if err := storage.MVCCPut(ctx, db, nil, roachpb.Key("unused1"), ts4, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } if err := db.Flush(); err != nil { @@ -92,7 +92,7 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { if _, err := storage.MVCCResolveWriteIntent(ctx, db, nil, intent); err != nil { t.Fatal(err) } - if err := storage.MVCCPut(ctx, db, nil, roachpb.Key("unused2"), ts1, v, nil); err != nil { + if err := storage.MVCCPut(ctx, db, nil, roachpb.Key("unused2"), ts1, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } if err := db.Flush(); err != nil { @@ -181,7 +181,7 @@ func TestRefreshRangeError(t *testing.T) { }, ReadTimestamp: ts2, } - if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, v, txn); err != nil { + if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, hlc.ClockTimestamp{}, v, txn); err != nil { t.Fatal(err) } @@ -246,7 +246,7 @@ func TestRefreshRangeTimestampBounds(t *testing.T) { ts3 := hlc.Timestamp{WallTime: 3} // Write to a key at time ts2. - require.NoError(t, storage.MVCCPut(ctx, db, nil, k, ts2, v, nil)) + require.NoError(t, storage.MVCCPut(ctx, db, nil, k, ts2, hlc.ClockTimestamp{}, v, nil)) for _, tc := range []struct { from, to hlc.Timestamp diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_test.go index 30a7bb8566b7..56ee5ee25394 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_test.go @@ -59,7 +59,7 @@ func TestRefreshError(t *testing.T) { }, ReadTimestamp: ts2, } - if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, v, txn); err != nil { + if err := storage.MVCCPut(ctx, db, nil, k, txn.ReadTimestamp, hlc.ClockTimestamp{}, v, txn); err != nil { t.Fatal(err) } @@ -120,7 +120,7 @@ func TestRefreshTimestampBounds(t *testing.T) { ts3 := hlc.Timestamp{WallTime: 3} // Write to a key at time ts2. - require.NoError(t, storage.MVCCPut(ctx, db, nil, k, ts2, v, nil)) + require.NoError(t, storage.MVCCPut(ctx, db, nil, k, ts2, hlc.ClockTimestamp{}, v, nil)) for _, tc := range []struct { from, to hlc.Timestamp diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go index ffc9023e4a2f..94d4e5ff6a98 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent_test.go @@ -167,13 +167,13 @@ func TestResolveIntentAfterPartialRollback(t *testing.T) { // Write a first value at key. v.SetString("a") txn.Sequence = 0 - if err := storage.MVCCPut(ctx, batch, nil, k, ts, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { t.Fatal(err) } // Write another value. v.SetString("b") txn.Sequence = 1 - if err := storage.MVCCPut(ctx, batch, nil, k, ts, v, &txn); err != nil { + if err := storage.MVCCPut(ctx, batch, nil, k, ts, hlc.ClockTimestamp{}, v, &txn); err != nil { t.Fatal(err) } if err := batch.Commit(true); err != nil { diff --git a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go index 8ba29e967fbb..5a2915488ddb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range_test.go @@ -95,7 +95,7 @@ func TestCmdRevertRange(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%04d", i)) var value roachpb.Value value.SetString(fmt.Sprintf("%d", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, baseTime.Add(int64(i%10), 0), value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, &stats, key, baseTime.Add(int64(i%10), 0), hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } } @@ -109,7 +109,7 @@ func TestCmdRevertRange(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%04d", i)) var value roachpb.Value value.SetString(fmt.Sprintf("%d-rev-a", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, tsA.Add(int64(i%5), 1), value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, &stats, key, tsA.Add(int64(i%5), 1), hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } } @@ -122,7 +122,7 @@ func TestCmdRevertRange(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%04d", i)) var value roachpb.Value value.SetString(fmt.Sprintf("%d-rev-b", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, tsB.Add(1, int32(i%5)), value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, &stats, key, tsB.Add(1, int32(i%5)), hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } } @@ -195,7 +195,7 @@ func TestCmdRevertRange(t *testing.T) { txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, tsC, 1, 1) if err := storage.MVCCPut( - ctx, eng, &stats, []byte("0012"), tsC, roachpb.MakeValueFromBytes([]byte("i")), &txn, + ctx, eng, &stats, []byte("0012"), tsC, hlc.ClockTimestamp{}, roachpb.MakeValueFromBytes([]byte("i")), &txn, ); err != nil { t.Fatal(err) } @@ -206,7 +206,7 @@ func TestCmdRevertRange(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%04d", i)) var value roachpb.Value value.SetString(fmt.Sprintf("%d-rev-b", i)) - if err := storage.MVCCPut(ctx, eng, &stats, key, tsC.Add(10, int32(i%5)), value, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, &stats, key, tsC.Add(10, int32(i%5)), hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatalf("writing key %s: %+v", key, err) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_scan_test.go b/pkg/kv/kvserver/batcheval/cmd_scan_test.go index 0d02c88315ba..640b372d3bfb 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan_test.go @@ -98,7 +98,7 @@ func testScanReverseScanInner( // Write to k1 and k2. for _, k := range []roachpb.Key{k1, k2} { - err := storage.MVCCPut(ctx, eng, nil, k, ts, roachpb.MakeValueFromString("value-"+string(k)), nil) + err := storage.MVCCPut(ctx, eng, nil, k, ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value-"+string(k)), nil) require.NoError(t, err) } @@ -199,7 +199,7 @@ func TestScanReverseScanWholeRows(t *testing.T) { for r := 0; r < 2; r++ { for cf := uint32(0); cf < 3; cf++ { key := makeRowKey(t, r, cf) - err := storage.MVCCPut(ctx, eng, nil, key, ts, roachpb.MakeValueFromString("value"), nil) + err := storage.MVCCPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value"), nil) require.NoError(t, err) rowKeys = append(rowKeys, key) } diff --git a/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go b/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go index 8bc4e3dcc15b..481d9af8c716 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log_test.go @@ -31,7 +31,7 @@ func putTruncatedState( key := keys.RaftTruncatedStateKey(rangeID) if err := storage.MVCCPutProto( context.Background(), eng, nil, key, - hlc.Timestamp{}, nil /* txn */, &truncState, + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &truncState, ); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/batcheval/intent_test.go b/pkg/kv/kvserver/batcheval/intent_test.go index f287e11da403..16d997395b7a 100644 --- a/pkg/kv/kvserver/batcheval/intent_test.go +++ b/pkg/kv/kvserver/batcheval/intent_test.go @@ -129,9 +129,9 @@ func TestCollectIntentsUsesSameIterator(t *testing.T) { txn := roachpb.MakeTransaction("test", key, roachpb.NormalUserPriority, ts, 0, 1) var err error if delete { - err = storage.MVCCDelete(ctx, db, nil, key, ts, &txn) + err = storage.MVCCDelete(ctx, db, nil, key, ts, hlc.ClockTimestamp{}, &txn) } else { - err = storage.MVCCPut(ctx, db, nil, key, ts, val, &txn) + err = storage.MVCCPut(ctx, db, nil, key, ts, hlc.ClockTimestamp{}, val, &txn) } require.NoError(t, err) diff --git a/pkg/kv/kvserver/batcheval/transaction_test.go b/pkg/kv/kvserver/batcheval/transaction_test.go index b7481b86738d..2b63eab6f401 100644 --- a/pkg/kv/kvserver/batcheval/transaction_test.go +++ b/pkg/kv/kvserver/batcheval/transaction_test.go @@ -84,7 +84,7 @@ func TestUpdateAbortSpan(t *testing.T) { type evalFn func(storage.ReadWriter, EvalContext, *enginepb.MVCCStats) error addIntent := func(b storage.ReadWriter, _ EvalContext, ms *enginepb.MVCCStats) error { val := roachpb.MakeValueFromString("val") - return storage.MVCCPut(ctx, b, ms, intentKey, txn.ReadTimestamp, val, &txn) + return storage.MVCCPut(ctx, b, ms, intentKey, txn.ReadTimestamp, hlc.ClockTimestamp{}, val, &txn) } addPrevAbortSpanEntry := func(b storage.ReadWriter, rec EvalContext, ms *enginepb.MVCCStats) error { return UpdateAbortSpan(ctx, rec, b, ms, prevTxn.TxnMeta, true /* poison */) diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index d4a6b0d0e10a..36c1e2bd60c1 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -3842,7 +3842,9 @@ func TestStoreRangeMergeRaftSnapshot(t *testing.T) { } tombstoneKey := keys.RangeTombstoneKey(rangeID) tombstoneValue := &roachpb.RangeTombstone{NextReplicaID: math.MaxInt32} - if err := storage.MVCCBlindPutProto(context.Background(), &sst, nil, tombstoneKey, hlc.Timestamp{}, tombstoneValue, nil); err != nil { + if err := storage.MVCCBlindPutProto( + context.Background(), &sst, nil, tombstoneKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, tombstoneValue, nil, + ); err != nil { return err } err := sst.Finish() diff --git a/pkg/kv/kvserver/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index eb370481f5b8..951f071e7aee 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -416,7 +416,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { val.SetInt(42) diffTimestamp = ts.Clock().Now() if err := storage.MVCCPut( - context.Background(), store1.Engine(), nil, diffKey, diffTimestamp, val, nil, + context.Background(), store1.Engine(), nil, diffKey, diffTimestamp, hlc.ClockTimestamp{}, val, nil, ); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/gc/data_distribution_test.go b/pkg/kv/kvserver/gc/data_distribution_test.go index 097761727e06..f032b7eb120d 100644 --- a/pkg/kv/kvserver/gc/data_distribution_test.go +++ b/pkg/kv/kvserver/gc/data_distribution_test.go @@ -64,7 +64,7 @@ func (ds dataDistribution) setupTest( txn.WriteTimestamp = ts } err := storage.MVCCPut(ctx, eng, &ms, kv.Key.Key, ts, - roachpb.Value{RawBytes: kv.Value}, txn) + hlc.ClockTimestamp{}, roachpb.Value{RawBytes: kv.Value}, txn) require.NoError(t, err) } if !kv.Key.Timestamp.Less(maxTs) { diff --git a/pkg/kv/kvserver/gc/gc_test.go b/pkg/kv/kvserver/gc/gc_test.go index 4e216b0c8d1e..4cb02651962d 100644 --- a/pkg/kv/kvserver/gc/gc_test.go +++ b/pkg/kv/kvserver/gc/gc_test.go @@ -120,7 +120,7 @@ func TestIntentAgeThresholdSetting(t *testing.T) { WallTime: intentTs.Nanoseconds(), } txn := roachpb.MakeTransaction("txn", key, roachpb.NormalUserPriority, intentHlc, 1000, 0) - require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, value, &txn)) + require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, hlc.ClockTimestamp{}, value, &txn)) require.NoError(t, eng.Flush()) // Prepare test fixtures for GC run. @@ -172,7 +172,7 @@ func TestIntentCleanupBatching(t *testing.T) { txn := roachpb.MakeTransaction("txn", key, roachpb.NormalUserPriority, intentHlc, 1000, 0) for _, suffix := range objectKeys { key := []byte{prefix, suffix} - require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, value, &txn)) + require.NoError(t, storage.MVCCPut(ctx, eng, nil, key, intentHlc, hlc.ClockTimestamp{}, value, &txn)) } require.NoError(t, eng.Flush()) } diff --git a/pkg/kv/kvserver/loqrecovery/apply.go b/pkg/kv/kvserver/loqrecovery/apply.go index b5eb2310ebe5..b76450249dc0 100644 --- a/pkg/kv/kvserver/loqrecovery/apply.go +++ b/pkg/kv/kvserver/loqrecovery/apply.go @@ -257,7 +257,7 @@ func applyReplicaUpdate( // A crude form of the intent resolution process: abort the // transaction by deleting its record. txnKey := keys.TransactionKey(intent.Txn.Key, intent.Txn.ID) - if err := storage.MVCCDelete(ctx, readWriter, &ms, txnKey, hlc.Timestamp{}, nil); err != nil { + if err := storage.MVCCDelete(ctx, readWriter, &ms, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { return PrepareReplicaReport{}, err } update := roachpb.LockUpdate{ @@ -285,7 +285,8 @@ func applyReplicaUpdate( if err := storage.MVCCPutProto( ctx, readWriter, &ms, key, clock.Now(), - nil /* txn */, &newDesc); err != nil { + hlc.ClockTimestamp{}, nil /* txn */, &newDesc, + ); err != nil { return PrepareReplicaReport{}, err } report.Descriptor = newDesc diff --git a/pkg/kv/kvserver/loqrecovery/recovery_env_test.go b/pkg/kv/kvserver/loqrecovery/recovery_env_test.go index 06f42716a8fc..5a7b8a8492fe 100644 --- a/pkg/kv/kvserver/loqrecovery/recovery_env_test.go +++ b/pkg/kv/kvserver/loqrecovery/recovery_env_test.go @@ -206,8 +206,9 @@ func (e *quorumRecoveryEnv) handleReplicationData(t *testing.T, d datadriven.Tes buildReplicaDescriptorFromTestData(t, replica) eng := e.getOrCreateStore(ctx, t, replica.StoreID, replica.NodeID) - if err = storage.MVCCPutProto(ctx, eng, nil, key, clock.Now(), nil, /* txn */ - &desc); err != nil { + if err = storage.MVCCPutProto( + ctx, eng, nil, key, clock.Now(), hlc.ClockTimestamp{}, nil /* txn */, &desc, + ); err != nil { t.Fatalf("failed to write range descriptor into store: %v", err) } @@ -426,8 +427,8 @@ func (e *quorumRecoveryEnv) getOrCreateStore( StoreID: storeID, } if err = storage.MVCCPutProto( - context.Background(), eng, nil, keys.StoreIdentKey(), hlc.Timestamp{}, nil, - &sIdent); err != nil { + context.Background(), eng, nil, keys.StoreIdentKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &sIdent, + ); err != nil { t.Fatalf("failed to populate test store ident: %v", err) } wrapped.engine = eng diff --git a/pkg/kv/kvserver/mvcc_gc_queue_test.go b/pkg/kv/kvserver/mvcc_gc_queue_test.go index 9ae102908a26..b44d3c031a52 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -282,7 +282,7 @@ func (cws *cachedWriteSimulator) multiKey( ts := hlc.Timestamp{}.Add(ms.LastUpdateNanos, 0) key, value := []byte("multikey"), cws.value(size) var eachMS enginepb.MVCCStats - if err := storage.MVCCPut(ctx, eng, &eachMS, key, ts, value, txn); err != nil { + if err := storage.MVCCPut(ctx, eng, &eachMS, key, ts, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } for i := 1; i < numOps; i++ { @@ -311,7 +311,7 @@ func (cws *cachedWriteSimulator) singleKeySteady( for i := 0; i < qps; i++ { now := initialNow.Add(elapsed.Nanoseconds(), int32(i)) - if err := storage.MVCCPut(ctx, eng, ms, key, now, value, nil /* txn */); err != nil { + if err := storage.MVCCPut(ctx, eng, ms, key, now, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } if len(firstSl) < cacheFirstLen { @@ -855,7 +855,7 @@ func TestMVCCGCQueueTransactionTable(t *testing.T) { txns[strKey] = *txn for _, addrKey := range []roachpb.Key{baseKey, outsideKey} { key := keys.TransactionKey(addrKey, txn.ID) - if err := storage.MVCCPutProto(ctx, tc.engine, nil, key, hlc.Timestamp{}, nil, txn); err != nil { + if err := storage.MVCCPutProto(ctx, tc.engine, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, txn); err != nil { t.Fatal(err) } } @@ -1057,7 +1057,7 @@ func TestMVCCGCQueueLastProcessedTimestamps(t *testing.T) { ts := tc.Clock().Now() for _, lpv := range lastProcessedVals { - if err := storage.MVCCPutProto(ctx, tc.engine, nil, lpv.key, hlc.Timestamp{}, nil, &ts); err != nil { + if err := storage.MVCCPutProto(ctx, tc.engine, nil, lpv.key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &ts); err != nil { t.Fatal(err) } } diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go index 7d13f32a92d5..428a6ef0dd16 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go @@ -262,7 +262,7 @@ func setupData( value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, opts.valueBytes)) value.InitChecksum(key) ts := hlc.Timestamp{WallTime: int64((pos + 1) * 5)} - if err := storage.MVCCPut(ctx, batch, nil /* ms */, key, ts, value, nil); err != nil { + if err := storage.MVCCPut(ctx, batch, nil /* ms */, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatal(err) } } diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go index 62bc973d3c0b..184b509aa6ee 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go @@ -83,12 +83,12 @@ func TestCatchupScan(t *testing.T) { // Put with no intent. for _, kv := range []storage.MVCCKeyValue{kv1_1_1, kv1_2_2, kv1_3_3, kv2_1_1, kv2_2_2, kv2_5_3} { v := roachpb.Value{RawBytes: kv.Value} - if err := storage.MVCCPut(ctx, eng, nil, kv.Key.Key, kv.Key.Timestamp, v, nil); err != nil { + if err := storage.MVCCPut(ctx, eng, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } // Put with an intent. - if err := storage.MVCCPut(ctx, eng, nil, kv1_4_4.Key.Key, txn.ReadTimestamp, val, &txn); err != nil { + if err := storage.MVCCPut(ctx, eng, nil, kv1_4_4.Key.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val, &txn); err != nil { t.Fatal(err) } testutils.RunTrueAndFalse(t, "useTBI", func(t *testing.T, useTBI bool) { diff --git a/pkg/kv/kvserver/rangefeed/task_test.go b/pkg/kv/kvserver/rangefeed/task_test.go index 87b695a7a228..9c87c4f93e40 100644 --- a/pkg/kv/kvserver/rangefeed/task_test.go +++ b/pkg/kv/kvserver/rangefeed/task_test.go @@ -266,7 +266,7 @@ func TestInitResolvedTSScan(t *testing.T) { } for _, op := range testData { kv := op.kv - err := storage.MVCCPut(ctx, engine, nil, kv.Key.Key, kv.Key.Timestamp, roachpb.Value{RawBytes: kv.Value}, op.txn) + err := storage.MVCCPut(ctx, engine, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, roachpb.Value{RawBytes: kv.Value}, op.txn) require.NoError(t, err) } return engine diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index 8ae5efa1001d..eaedadbafaaa 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -104,7 +104,7 @@ func createRangeData( allKeys := []storage.MVCCKey{} for _, keyTS := range keyTSs { - if err := storage.MVCCPut(context.Background(), eng, nil, keyTS.key, keyTS.ts, roachpb.MakeValueFromString("value"), nil); err != nil { + if err := storage.MVCCPut(context.Background(), eng, nil, keyTS.key, keyTS.ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value"), nil); err != nil { t.Fatal(err) } allKeys = append(allKeys, storage.MVCCKey{Key: keyTS.key, Timestamp: keyTS.ts}) diff --git a/pkg/kv/kvserver/readsummary/persist.go b/pkg/kv/kvserver/readsummary/persist.go index 9c03878591ac..bde7509f6e33 100644 --- a/pkg/kv/kvserver/readsummary/persist.go +++ b/pkg/kv/kvserver/readsummary/persist.go @@ -44,5 +44,5 @@ func Set( sum *rspb.ReadSummary, ) error { key := keys.RangePriorReadSummaryKey(rangeID) - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, nil, sum) + return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, sum) } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 49edaace9871..94224e199ce5 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1174,7 +1174,8 @@ func (r *Replica) GetLastReplicaGCTimestamp(ctx context.Context) (hlc.Timestamp, func (r *Replica) setLastReplicaGCTimestamp(ctx context.Context, timestamp hlc.Timestamp) error { key := keys.RangeLastReplicaGCTimestampKey(r.RangeID) - return storage.MVCCPutProto(ctx, r.store.Engine(), nil, key, hlc.Timestamp{}, nil, ×tamp) + return storage.MVCCPutProto( + ctx, r.store.Engine(), nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, ×tamp) } // getQueueLastProcessed returns the last processed timestamp for the diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index a6a8ed9fd58f..ba9518b08d3f 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -249,5 +249,5 @@ func writeTombstoneKey( } // "Blind" because ms == nil and timestamp.IsEmpty(). return storage.MVCCBlindPutProto(ctx, writer, nil, tombstoneKey, - hlc.Timestamp{}, tombstone, nil) + hlc.Timestamp{}, hlc.ClockTimestamp{}, tombstone, nil) } diff --git a/pkg/kv/kvserver/replica_evaluate_test.go b/pkg/kv/kvserver/replica_evaluate_test.go index fb1ce01672f9..dd94e385702f 100644 --- a/pkg/kv/kvserver/replica_evaluate_test.go +++ b/pkg/kv/kvserver/replica_evaluate_test.go @@ -712,7 +712,7 @@ func writeABCDEFIntents(t *testing.T, d *data, txn *roachpb.Transaction) { func writeABCDEFWith(t *testing.T, eng storage.Engine, ts hlc.Timestamp, txn *roachpb.Transaction) { for _, k := range []string{"a", "b", "c", "d", "e", "f"} { require.NoError(t, storage.MVCCPut( - context.Background(), eng, nil /* ms */, roachpb.Key(k), ts, + context.Background(), eng, nil /* ms */, roachpb.Key(k), ts, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("value-"+k), txn)) } } diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 8754d90acdca..2a171cc88508 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -2116,8 +2116,14 @@ func handleTruncatedStateBelowRaftPreApply( // The suggested truncated state moves us forward; apply it and tell // the caller as much. if err := storage.MVCCPutProto( - ctx, readWriter, nil /* ms */, prefixBuf.RaftTruncatedStateKey(), - hlc.Timestamp{}, nil /* txn */, suggestedTruncatedState, + ctx, + readWriter, + nil, /* ms */ + prefixBuf.RaftTruncatedStateKey(), + hlc.Timestamp{}, + hlc.ClockTimestamp{}, + nil, /* txn */ + suggestedTruncatedState, ); err != nil { return false, errors.Wrap(err, "unable to write RaftTruncatedState") } diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index c27522be587c..5da1475c1448 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -661,7 +661,7 @@ func (r *Replica) append( value.InitChecksum(key) var err error if ent.Index > prevLastIndex { - err = storage.MVCCBlindPut(ctx, writer, &diff, key, hlc.Timestamp{}, value, nil /* txn */) + err = storage.MVCCBlindPut(ctx, writer, &diff, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, value, nil /* txn */) } else { // We type assert `writer` to also be an engine.ReadWriter only in // the case where we're replacing existing entries. @@ -669,7 +669,7 @@ func (r *Replica) append( if !ok { panic("expected writer to be a engine.ReadWriter when overwriting log entries") } - err = storage.MVCCPut(ctx, eng, &diff, key, hlc.Timestamp{}, value, nil /* txn */) + err = storage.MVCCPut(ctx, eng, &diff, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, value, nil /* txn */) } if err != nil { return 0, 0, 0, err @@ -690,7 +690,7 @@ func (r *Replica) append( // Note that the caller is in charge of deleting any sideloaded payloads // (which they must only do *after* the batch has committed). err := storage.MVCCDelete(ctx, eng, &diff, r.raftMu.stateLoader.RaftLogKey(i), - hlc.Timestamp{}, nil /* txn */) + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil) if err != nil { return 0, 0, 0, err } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index a30aac136395..90acce579380 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -1191,7 +1191,7 @@ func TestReplicaGossipConfigsOnLease(t *testing.T) { key := keys.SystemSQLCodec.TablePrefix(keys.MaxSystemConfigDescID) var val roachpb.Value val.SetInt(42) - if err := storage.MVCCPut(context.Background(), tc.engine, nil, key, hlc.Timestamp{}, val, nil); err != nil { + if err := storage.MVCCPut(context.Background(), tc.engine, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } @@ -1989,7 +1989,7 @@ func TestOptimizePuts(t *testing.T) { for i, c := range testCases { if c.exKey != nil { if err := storage.MVCCPut(context.Background(), tc.engine, nil, c.exKey, - hlc.Timestamp{}, roachpb.MakeValueFromString("foo"), nil); err != nil { + hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("foo"), nil); err != nil { t.Fatal(err) } } @@ -3370,7 +3370,7 @@ func TestReplicaAbortSpanReadError(t *testing.T) { // Overwrite Abort span entry with garbage for the last op. key := keys.AbortSpanKey(tc.repl.RangeID, txn.ID) - err := storage.MVCCPut(ctx, tc.engine, nil, key, hlc.Timestamp{}, roachpb.MakeValueFromString("never read in this test"), nil) + err := storage.MVCCPut(ctx, tc.engine, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("never read in this test"), nil) if err != nil { t.Fatal(err) } @@ -4452,7 +4452,7 @@ func TestEndTxnWithErrors(t *testing.T) { existTxnRecord := existTxn.AsRecord() txnKey := keys.TransactionKey(test.key, txn.ID) if err := storage.MVCCPutProto( - ctx, tc.repl.store.Engine(), nil, txnKey, hlc.Timestamp{}, nil, &existTxnRecord, + ctx, tc.repl.store.Engine(), nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &existTxnRecord, ); err != nil { t.Fatal(err) } @@ -4495,7 +4495,7 @@ func TestEndTxnWithErrorAndSyncIntentResolution(t *testing.T) { existTxn.Status = roachpb.ABORTED existTxnRec := existTxn.AsRecord() txnKey := keys.TransactionKey(txn.Key, txn.ID) - err := storage.MVCCPutProto(ctx, tc.repl.store.Engine(), nil, txnKey, hlc.Timestamp{}, nil, &existTxnRec) + err := storage.MVCCPutProto(ctx, tc.repl.store.Engine(), nil, txnKey, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &existTxnRec) require.NoError(t, err) // End the transaction, verify expected error, shouldn't deadlock. @@ -7040,7 +7040,7 @@ func TestReplicaLoadSystemConfigSpanIntent(t *testing.T) { v := roachpb.MakeValueFromString("foo") testutils.SucceedsSoon(t, func() error { if err := storage.MVCCPut(ctx, repl.store.Engine(), &enginepb.MVCCStats{}, - keys.SystemConfigSpan.Key, repl.store.Clock().Now(), v, nil); err != nil { + keys.SystemConfigSpan.Key, repl.store.Clock().Now(), hlc.ClockTimestamp{}, v, nil); err != nil { return err } @@ -11080,7 +11080,7 @@ func TestReplicaPushed1PC(t *testing.T) { // Write a value outside the transaction. tc.manualClock.Increment(10) ts2 := tc.Clock().Now() - if err := storage.MVCCPut(ctx, tc.engine, nil, k, ts2, roachpb.MakeValueFromString("one"), nil); err != nil { + if err := storage.MVCCPut(ctx, tc.engine, nil, k, ts2, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("one"), nil); err != nil { t.Fatalf("writing interfering value: %+v", err) } @@ -13482,7 +13482,7 @@ func setMockPutWithEstimates(containsEstimatesDelta int64) (undo func()) { ms := cArgs.Stats ms.ContainsEstimates += containsEstimatesDelta ts := cArgs.Header.Timestamp - return result.Result{}, storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, args.Value, cArgs.Header.Txn) + return result.Result{}, storage.MVCCBlindPut(ctx, readWriter, ms, args.Key, ts, hlc.ClockTimestamp{}, args.Value, cArgs.Header.Txn) } batcheval.UnregisterCommand(roachpb.Put) diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index 9769a25a4d90..ecabcd9fe8bd 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -158,7 +158,7 @@ func (rsl StateLoader) SetLease( ctx context.Context, readWriter storage.ReadWriter, ms *enginepb.MVCCStats, lease roachpb.Lease, ) error { return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeLeaseKey(), - hlc.Timestamp{}, nil, &lease) + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &lease) } // LoadRangeAppliedState loads the Range applied state. @@ -213,7 +213,8 @@ func (rsl StateLoader) SetRangeAppliedState( // The RangeAppliedStateKey is not included in stats. This is also reflected // in C.MVCCComputeStats and ComputeStatsForRange. ms := (*enginepb.MVCCStats)(nil) - return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeAppliedStateKey(), hlc.Timestamp{}, nil, &as) + return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeAppliedStateKey(), + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &as) } // SetMVCCStats overwrites the MVCC stats. This needs to perform a read on the @@ -264,8 +265,8 @@ func (rsl StateLoader) SetGCThreshold( if threshold == nil { return errors.New("cannot persist nil GCThreshold") } - return storage.MVCCPutProto(ctx, readWriter, ms, - rsl.RangeGCThresholdKey(), hlc.Timestamp{}, nil, threshold) + return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeGCThresholdKey(), + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, threshold) } // LoadVersion loads the replica version. @@ -285,8 +286,8 @@ func (rsl StateLoader) SetVersion( ms *enginepb.MVCCStats, version *roachpb.Version, ) error { - return storage.MVCCPutProto(ctx, readWriter, ms, - rsl.RangeVersionKey(), hlc.Timestamp{}, nil, version) + return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeVersionKey(), + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, version) } // The rest is not technically part of ReplicaState. @@ -347,7 +348,8 @@ func (rsl StateLoader) SetRaftTruncatedState( writer, nil, /* ms */ rsl.RaftTruncatedStateKey(), - hlc.Timestamp{}, /* timestamp */ + hlc.Timestamp{}, /* timestamp */ + hlc.ClockTimestamp{}, /* localTimestamp */ truncState, nil, /* txn */ ) @@ -377,7 +379,8 @@ func (rsl StateLoader) SetHardState( writer, nil, /* ms */ rsl.RaftHardStateKey(), - hlc.Timestamp{}, /* timestamp */ + hlc.Timestamp{}, /* timestamp */ + hlc.ClockTimestamp{}, /* localTimestamp */ &hs, nil, /* txn */ ) @@ -451,7 +454,8 @@ func (rsl StateLoader) SetRaftReplicaID( writer, nil, /* ms */ rsl.RaftReplicaIDKey(), - hlc.Timestamp{}, /* timestamp */ + hlc.Timestamp{}, /* timestamp */ + hlc.ClockTimestamp{}, /* localTimestamp */ &rid, nil, /* txn */ ) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index caab991c5a05..2bc90dd9786d 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -2662,6 +2662,7 @@ func (s *Store) WriteLastUpTimestamp(ctx context.Context, time hlc.Timestamp) er nil, keys.StoreLastUpKey(), hlc.Timestamp{}, + hlc.ClockTimestamp{}, nil, &time, ) @@ -2697,6 +2698,7 @@ func (s *Store) WriteHLCUpperBound(ctx context.Context, time int64) error { nil, keys.StoreHLCUpperBoundKey(), hlc.Timestamp{}, + hlc.ClockTimestamp{}, nil, &ts, ); err != nil { @@ -3597,7 +3599,16 @@ func (s *storeForTruncatorImpl) getEngine() storage.Engine { func WriteClusterVersion( ctx context.Context, eng storage.Engine, cv clusterversion.ClusterVersion, ) error { - err := storage.MVCCPutProto(ctx, eng, nil, keys.StoreClusterVersionKey(), hlc.Timestamp{}, nil, &cv) + err := storage.MVCCPutProto( + ctx, + eng, + nil, + keys.StoreClusterVersionKey(), + hlc.Timestamp{}, + hlc.ClockTimestamp{}, + nil, + &cv, + ) if err != nil { return err } diff --git a/pkg/kv/kvserver/store_init.go b/pkg/kv/kvserver/store_init.go index 321c8fc21ac1..4cbd9f971021 100644 --- a/pkg/kv/kvserver/store_init.go +++ b/pkg/kv/kvserver/store_init.go @@ -58,6 +58,7 @@ func InitEngine(ctx context.Context, eng storage.Engine, ident roachpb.StoreIden nil, keys.StoreIdentKey(), hlc.Timestamp{}, + hlc.ClockTimestamp{}, nil, &ident, ); err != nil { @@ -203,7 +204,7 @@ func WriteInitialClusterData( // Range descriptor. if err := storage.MVCCPutProto( ctx, batch, nil /* ms */, keys.RangeDescriptorKey(desc.StartKey), - now, nil /* txn */, desc, + now, hlc.ClockTimestamp{}, nil /* txn */, desc, ); err != nil { return err } @@ -211,14 +212,15 @@ func WriteInitialClusterData( // Replica GC timestamp. if err := storage.MVCCPutProto( ctx, batch, nil /* ms */, keys.RangeLastReplicaGCTimestampKey(desc.RangeID), - hlc.Timestamp{}, nil /* txn */, &now, + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &now, ); err != nil { return err } // Range addressing for meta2. meta2Key := keys.RangeMetaKey(endKey) - if err := storage.MVCCPutProto(ctx, batch, firstRangeMS, meta2Key.AsRawKey(), - now, nil /* txn */, desc, + if err := storage.MVCCPutProto( + ctx, batch, firstRangeMS, meta2Key.AsRawKey(), + now, hlc.ClockTimestamp{}, nil /* txn */, desc, ); err != nil { return err } @@ -228,7 +230,7 @@ func WriteInitialClusterData( // Range addressing for meta1. meta1Key := keys.RangeMetaKey(keys.RangeMetaKey(roachpb.RKeyMax)) if err := storage.MVCCPutProto( - ctx, batch, nil /* ms */, meta1Key.AsRawKey(), now, nil /* txn */, desc, + ctx, batch, nil /* ms */, meta1Key.AsRawKey(), now, hlc.ClockTimestamp{}, nil /* txn */, desc, ); err != nil { return err } @@ -239,7 +241,7 @@ func WriteInitialClusterData( // Initialize the checksums. kv.Value.InitChecksum(kv.Key) if err := storage.MVCCPut( - ctx, batch, nil /* ms */, kv.Key, now, kv.Value, nil, /* txn */ + ctx, batch, nil /* ms */, kv.Key, now, hlc.ClockTimestamp{}, kv.Value, nil, /* txn */ ); err != nil { return err } diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 16206dcc9650..c383e9a98e23 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -1020,7 +1020,7 @@ func SendEmptySnapshot( var ms enginepb.MVCCStats // Seed an empty range into the new engine. if err := storage.MVCCPutProto( - ctx, eng, &ms, keys.RangeDescriptorKey(desc.StartKey), now, nil /* txn */, &desc, + ctx, eng, &ms, keys.RangeDescriptorKey(desc.StartKey), now, hlc.ClockTimestamp{}, nil /* txn */, &desc, ); err != nil { return err } diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index aa345f03be90..2cb131e961f6 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -337,6 +337,7 @@ func TestIterateIDPrefixKeys(t *testing.T) { nil, /* ms */ key, hlc.Timestamp{}, + hlc.ClockTimestamp{}, roachpb.MakeValueFromString("fake value for "+key.String()), nil, /* txn */ ); err != nil { @@ -370,7 +371,7 @@ func TestIterateIDPrefixKeys(t *testing.T) { t.Logf("writing tombstone at rangeID=%d", rangeID) if err := storage.MVCCPutProto( - ctx, eng, nil /* ms */, keys.RangeTombstoneKey(rangeID), hlc.Timestamp{}, nil /* txn */, &tombstone, + ctx, eng, nil /* ms */, keys.RangeTombstoneKey(rangeID), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &tombstone, ); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/stores.go b/pkg/kv/kvserver/stores.go index 9d4bf5f5c9e4..5a9414a36f79 100644 --- a/pkg/kv/kvserver/stores.go +++ b/pkg/kv/kvserver/stores.go @@ -282,7 +282,7 @@ func (ls *Stores) updateBootstrapInfoLocked(bi *gossip.BootstrapInfo) error { var err error ls.storeMap.Range(func(k int64, v unsafe.Pointer) bool { s := (*Store)(v) - err = storage.MVCCPutProto(ctx, s.engine, nil, keys.StoreGossipKey(), hlc.Timestamp{}, nil, bi) + err = storage.MVCCPutProto(ctx, s.engine, nil, keys.StoreGossipKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, bi) return err == nil }) return err diff --git a/pkg/kv/kvserver/txn_wait_queue_test.go b/pkg/kv/kvserver/txn_wait_queue_test.go index c24e343626c2..4526725fb97e 100644 --- a/pkg/kv/kvserver/txn_wait_queue_test.go +++ b/pkg/kv/kvserver/txn_wait_queue_test.go @@ -39,7 +39,7 @@ import ( func writeTxnRecord(ctx context.Context, tc *testContext, txn *roachpb.Transaction) error { key := keys.TransactionKey(txn.Key, txn.ID) - return storage.MVCCPutProto(ctx, tc.store.Engine(), nil, key, hlc.Timestamp{}, nil, txn) + return storage.MVCCPutProto(ctx, tc.store.Engine(), nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, txn) } // createTxnForPushQueue creates a txn struct and writes a "fake" diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index 56e3d1d243ca..85d7c19b14b4 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -265,7 +265,7 @@ func TestCorruptedClusterID(t *testing.T) { StoreID: 1, } if err := storage.MVCCPutProto( - ctx, e, nil /* ms */, keys.StoreIdentKey(), hlc.Timestamp{}, nil /* txn */, &sIdent, + ctx, e, nil /* ms */, keys.StoreIdentKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil /* txn */, &sIdent, ); err != nil { t.Fatal(err) } diff --git a/pkg/server/node_tombstone_storage.go b/pkg/server/node_tombstone_storage.go index caa517d6a472..e009366f31a6 100644 --- a/pkg/server/node_tombstone_storage.go +++ b/pkg/server/node_tombstone_storage.go @@ -136,7 +136,7 @@ func (s *nodeTombstoneStorage) SetDecommissioned( } if err := storage.MVCCPut( - ctx, eng, nil /* MVCCStats */, k, hlc.Timestamp{}, v, nil, /* txn */ + ctx, eng, nil /* ms */, k, hlc.Timestamp{}, hlc.ClockTimestamp{}, v, nil, /* txn */ ); err != nil { return err } diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index bfb3f875d3fa..520d26de582f 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -569,7 +569,8 @@ func TestClusterIDMismatch(t *testing.T) { StoreID: roachpb.StoreID(i + 1), } if err := storage.MVCCPutProto( - context.Background(), e, nil, keys.StoreIdentKey(), hlc.Timestamp{}, nil, &sIdent); err != nil { + context.Background(), e, nil, keys.StoreIdentKey(), hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &sIdent, + ); err != nil { t.Fatal(err) } @@ -1160,7 +1161,7 @@ func TestAssertEnginesEmpty(t *testing.T) { require.NoError(t, assertEnginesEmpty([]storage.Engine{eng})) require.NoError(t, storage.MVCCPutProto(ctx, eng, nil, keys.StoreClusterVersionKey(), - hlc.Timestamp{}, nil, &roachpb.Version{Major: 21, Minor: 1, Internal: 122})) + hlc.Timestamp{}, hlc.ClockTimestamp{}, nil, &roachpb.Version{Major: 21, Minor: 1, Internal: 122})) require.NoError(t, assertEnginesEmpty([]storage.Engine{eng})) batch := eng.NewBatch() diff --git a/pkg/server/settings_cache.go b/pkg/server/settings_cache.go index 7a0196572838..d4320c572cf6 100644 --- a/pkg/server/settings_cache.go +++ b/pkg/server/settings_cache.go @@ -100,7 +100,7 @@ func storeCachedSettingsKVs(ctx context.Context, eng storage.Engine, kvs []roach for _, kv := range kvs { kv.Value.Timestamp = hlc.Timestamp{} // nb: Timestamp is not part of checksum if err := storage.MVCCPut( - ctx, batch, nil, keys.StoreCachedSettingsKey(kv.Key), hlc.Timestamp{}, kv.Value, nil, + ctx, batch, nil, keys.StoreCachedSettingsKey(kv.Key), hlc.Timestamp{}, hlc.ClockTimestamp{}, kv.Value, nil, ); err != nil { return err } diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index 7a540849a7c2..34f294f02cbc 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -265,10 +265,10 @@ func BenchmarkMVCCPutDelete_Pebble(b *testing.B) { key := encoding.EncodeVarintAscending(nil, blockID) key = encoding.EncodeVarintAscending(key, blockNum) - if err := MVCCPut(ctx, db, nil, key, hlc.Timestamp{}, value, nil /* txn */); err != nil { + if err := MVCCPut(ctx, db, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatal(err) } - if err := MVCCDelete(ctx, db, nil, key, hlc.Timestamp{}, nil /* txn */); err != nil { + if err := MVCCDelete(ctx, db, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { b.Fatal(err) } } diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index f9b46600afd3..7c848a84dd8b 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -244,7 +244,7 @@ func setupKeysWithIntent( putTxn = &otherTxn } key := makeKey(nil, j) - require.NoError(b, MVCCPut(context.Background(), batch, nil, key, ts, value, putTxn)) + require.NoError(b, MVCCPut(context.Background(), batch, nil, key, ts, hlc.ClockTimestamp{}, value, putTxn)) } require.NoError(b, batch.Commit(true)) batch.Close() @@ -657,7 +657,7 @@ func loadTestData(dir string, numKeys, numBatches, batchTimeSpan, valueBytes int timestamp := hlc.Timestamp{WallTime: minWallTime + rand.Int63n(int64(batchTimeSpan))} value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, valueBytes)) value.InitChecksum(key) - if err := MVCCPut(ctx, batch, nil, key, timestamp, value, nil); err != nil { + if err := MVCCPut(ctx, batch, nil, key, timestamp, hlc.ClockTimestamp{}, value, nil); err != nil { return nil, err } } @@ -755,7 +755,7 @@ func setupMVCCData( txn.ReadTimestamp = ts txn.WriteTimestamp = ts } - if err := MVCCPut(ctx, batch, nil /* ms */, key, ts, value, txn); err != nil { + if err := MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, txn); err != nil { b.Fatal(err) } } @@ -957,7 +957,7 @@ func runMVCCPut(ctx context.Context, b *testing.B, emk engineMaker, valueSize in for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCPut(ctx, eng, nil, key, ts, value, nil); err != nil { + if err := MVCCPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -979,7 +979,7 @@ func runMVCCBlindPut(ctx context.Context, b *testing.B, emk engineMaker, valueSi for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCBlindPut(ctx, eng, nil, key, ts, value, nil); err != nil { + if err := MVCCBlindPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1003,7 +1003,7 @@ func runMVCCConditionalPut( for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCPut(ctx, eng, nil, key, ts, value, nil); err != nil { + if err := MVCCPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1015,7 +1015,7 @@ func runMVCCConditionalPut( for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCConditionalPut(ctx, eng, nil, key, ts, value, expected, CPutFailIfMissing, nil); err != nil { + if err := MVCCConditionalPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, expected, CPutFailIfMissing, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1037,7 +1037,9 @@ func runMVCCBlindConditionalPut(ctx context.Context, b *testing.B, emk engineMak for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCBlindConditionalPut(ctx, eng, nil, key, ts, value, nil, CPutFailIfMissing, nil); err != nil { + if err := MVCCBlindConditionalPut( + ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, nil, CPutFailIfMissing, nil, + ); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1059,7 +1061,7 @@ func runMVCCInitPut(ctx context.Context, b *testing.B, emk engineMaker, valueSiz for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCInitPut(ctx, eng, nil, key, ts, value, false, nil); err != nil { + if err := MVCCInitPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, false, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1081,7 +1083,7 @@ func runMVCCBlindInitPut(ctx context.Context, b *testing.B, emk engineMaker, val for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCBlindInitPut(ctx, eng, nil, key, ts, value, false, nil); err != nil { + if err := MVCCBlindInitPut(ctx, eng, nil, key, ts, hlc.ClockTimestamp{}, value, false, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1111,7 +1113,7 @@ func runMVCCBatchPut(ctx context.Context, b *testing.B, emk engineMaker, valueSi for j := i; j < end; j++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(j))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCPut(ctx, batch, nil, key, ts, value, nil); err != nil { + if err := MVCCPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } } @@ -1260,6 +1262,7 @@ func runMVCCDeleteRange(ctx context.Context, b *testing.B, emk engineMaker, valu roachpb.KeyMax, math.MaxInt64, hlc.MaxTimestamp, + hlc.ClockTimestamp{}, nil, false, ); err != nil { @@ -1416,7 +1419,7 @@ func runMVCCGarbageCollect( }) } for j := 0; j < opts.numVersions; j++ { - if err := MVCCPut(ctx, batch, nil /* ms */, key, ts.Add(0, int32(j)), val, nil); err != nil { + if err := MVCCPut(ctx, batch, nil, key, ts.Add(0, int32(j)), hlc.ClockTimestamp{}, val, nil); err != nil { b.Fatal(err) } } @@ -1472,7 +1475,7 @@ func runBatchApplyBatchRepr( for i := 0; i < batchSize; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(order[i]))) ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - if err := MVCCBlindPut(ctx, batch, nil, key, ts, value, nil); err != nil { + if err := MVCCBlindPut(ctx, batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatal(err) } } diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index b917225ec2af..cc8653983721 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -178,15 +178,13 @@ func TestEngineBatchStaleCachedIterator(t *testing.T) { // Put a value so that the deletion below finds a value to seek // to. - if err := MVCCPut(context.Background(), batch, nil, key, hlc.Timestamp{}, - roachpb.MakeValueFromString("x"), nil); err != nil { + if err := MVCCPut(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("x"), nil); err != nil { t.Fatal(err) } // Seek the iterator to `key` and clear the value (but without // telling the iterator about that). - if err := MVCCDelete(context.Background(), batch, nil, key, - hlc.Timestamp{}, nil); err != nil { + if err := MVCCDelete(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -1647,7 +1645,7 @@ func TestScanIntents(t *testing.T) { defer eng.Close() for _, key := range keys { - err := MVCCPut(ctx, eng, nil, key, txn1.ReadTimestamp, roachpb.Value{RawBytes: key}, txn1) + err := MVCCPut(ctx, eng, nil, key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, roachpb.Value{RawBytes: key}, txn1) require.NoError(t, err) } diff --git a/pkg/storage/metamorphic/operations.go b/pkg/storage/metamorphic/operations.go index 9e52e9b5aee4..6773db820d67 100644 --- a/pkg/storage/metamorphic/operations.go +++ b/pkg/storage/metamorphic/operations.go @@ -206,7 +206,7 @@ func (m mvccPutOp) run(ctx context.Context) string { txn.Sequence++ writer := m.m.getReadWriter(m.writer) - err := storage.MVCCPut(ctx, writer, nil, m.key, txn.WriteTimestamp, m.value, txn) + err := storage.MVCCPut(ctx, writer, nil, m.key, txn.WriteTimestamp, hlc.ClockTimestamp{}, m.value, txn) if err != nil { return fmt.Sprintf("error: %s", err) } @@ -229,7 +229,8 @@ func (m mvccCPutOp) run(ctx context.Context) string { writer := m.m.getReadWriter(m.writer) txn.Sequence++ - err := storage.MVCCConditionalPut(ctx, writer, nil, m.key, txn.WriteTimestamp, m.value, m.expVal, true, txn) + err := storage.MVCCConditionalPut(ctx, writer, nil, m.key, + txn.WriteTimestamp, hlc.ClockTimestamp{}, m.value, m.expVal, true, txn) if err != nil { return fmt.Sprintf("error: %s", err) } @@ -251,7 +252,7 @@ func (m mvccInitPutOp) run(ctx context.Context) string { writer := m.m.getReadWriter(m.writer) txn.Sequence++ - err := storage.MVCCInitPut(ctx, writer, nil, m.key, txn.WriteTimestamp, m.value, false, txn) + err := storage.MVCCInitPut(ctx, writer, nil, m.key, txn.WriteTimestamp, hlc.ClockTimestamp{}, m.value, false, txn) if err != nil { return fmt.Sprintf("error: %s", err) } @@ -278,7 +279,8 @@ func (m mvccDeleteRangeOp) run(ctx context.Context) string { txn.Sequence++ - keys, _, _, err := storage.MVCCDeleteRange(ctx, writer, nil, m.key, m.endKey, 0, txn.WriteTimestamp, txn, true) + keys, _, _, err := storage.MVCCDeleteRange(ctx, writer, nil, m.key, m.endKey, + 0, txn.WriteTimestamp, hlc.ClockTimestamp{}, txn, true) if err != nil { return fmt.Sprintf("error: %s", err) } @@ -331,7 +333,7 @@ func (m mvccDeleteOp) run(ctx context.Context) string { writer := m.m.getReadWriter(m.writer) txn.Sequence++ - err := storage.MVCCDelete(ctx, writer, nil, m.key, txn.WriteTimestamp, txn) + err := storage.MVCCDelete(ctx, writer, nil, m.key, txn.WriteTimestamp, hlc.ClockTimestamp{}, txn) if err != nil { return fmt.Sprintf("error: %s", err) } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 55f9fe3074a0..c7c5ac51fbf6 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -636,6 +636,7 @@ func MVCCPutProto( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, msg protoutil.Message, ) error { @@ -644,7 +645,7 @@ func MVCCPutProto( return err } value.InitChecksum(key) - return MVCCPut(ctx, rw, ms, key, timestamp, value, txn) + return MVCCPut(ctx, rw, ms, key, timestamp, localTimestamp, value, txn) } // MVCCBlindPutProto sets the given key to the protobuf-serialized byte string @@ -656,6 +657,7 @@ func MVCCBlindPutProto( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, msg protoutil.Message, txn *roachpb.Transaction, ) error { @@ -664,7 +666,7 @@ func MVCCBlindPutProto( return err } value.InitChecksum(key) - return MVCCBlindPut(ctx, writer, ms, key, timestamp, value, txn) + return MVCCBlindPut(ctx, writer, ms, key, timestamp, localTimestamp, value, txn) } // MVCCGetOptions bundles options for the MVCCGet family of functions. @@ -987,6 +989,7 @@ func MVCCPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, txn *roachpb.Transaction, ) error { @@ -998,7 +1001,7 @@ func MVCCPut( iter = rw.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{Prefix: true}) defer iter.Close() } - return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, txn, nil /* valueFn */) + return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, value, txn, nil) } // MVCCBlindPut is a fast-path of MVCCPut. See the MVCCPut comments for details @@ -1017,10 +1020,11 @@ func MVCCBlindPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, txn *roachpb.Transaction, ) error { - return mvccPutUsingIter(ctx, writer, nil, ms, key, timestamp, value, txn, nil /* valueFn */) + return mvccPutUsingIter(ctx, writer, nil, ms, key, timestamp, localTimestamp, value, txn, nil) } // MVCCDelete marks the key deleted so that it will not be returned in @@ -1035,12 +1039,13 @@ func MVCCDelete( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, ) error { iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) defer iter.Close() - return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, noValue, txn, nil /* valueFn */) + return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, nil) } var noValue = roachpb.Value{} @@ -1056,6 +1061,7 @@ func mvccPutUsingIter( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, txn *roachpb.Transaction, valueFn func(optionalValue) ([]byte, error), @@ -1070,8 +1076,7 @@ func mvccPutUsingIter( buf := newPutBuffer() - err := mvccPutInternal(ctx, writer, iter, ms, key, timestamp, rawBytes, - txn, buf, valueFn) + err := mvccPutInternal(ctx, writer, iter, ms, key, timestamp, localTimestamp, rawBytes, txn, buf, valueFn) // Using defer would be more convenient, but it is measurably slower. buf.release() @@ -1262,6 +1267,15 @@ func replayTransactionalWrite( // read timestamp. (One could imagine instead requiring that the timestamp // parameter be set to hlc.Timestamp{} when writing transactionally, but // hlc.Timestamp{} is already used as a sentinel for inline puts.) +// +// The local timestamp parameter dictates the local clock timestamp +// assigned to the key-value. It should be taken from the local HLC +// clock on the leaseholder that is performing the write and must be +// below the leaseholder's lease expiration. If the supplied local +// timestamp is empty (hlc.ClockTimestamp{}), the value will not be +// assigned an explicit local timestamp. The effect of this is that +// readers treat the local timestamp as being equal to the version +// timestamp. func mvccPutInternal( ctx context.Context, writer Writer, @@ -1269,6 +1283,7 @@ func mvccPutInternal( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value []byte, txn *roachpb.Transaction, buf *putBuffer, @@ -1656,6 +1671,7 @@ func MVCCIncrement( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, inc int64, ) (int64, error) { @@ -1664,7 +1680,7 @@ func MVCCIncrement( var int64Val int64 var newInt64Val int64 - err := mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, noValue, txn, func(value optionalValue) ([]byte, error) { + valueFn := func(value optionalValue) ([]byte, error) { if value.IsPresent() { var err error if int64Val, err = value.GetInt(); err != nil { @@ -1688,7 +1704,8 @@ func MVCCIncrement( newValue.SetInt(newInt64Val) newValue.InitChecksum(key) return newValue.RawBytes, nil - }) + } + err := mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) return newInt64Val, err } @@ -1726,6 +1743,7 @@ func MVCCConditionalPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, expVal []byte, allowIfDoesNotExist CPutMissingBehavior, @@ -1734,7 +1752,8 @@ func MVCCConditionalPut( iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) defer iter.Close() - return mvccConditionalPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, expVal, allowIfDoesNotExist, txn) + return mvccConditionalPutUsingIter( + ctx, rw, iter, ms, key, timestamp, localTimestamp, value, expVal, allowIfDoesNotExist, txn) } // MVCCBlindConditionalPut is a fast-path of MVCCConditionalPut. See the @@ -1752,12 +1771,14 @@ func MVCCBlindConditionalPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, expVal []byte, allowIfDoesNotExist CPutMissingBehavior, txn *roachpb.Transaction, ) error { - return mvccConditionalPutUsingIter(ctx, writer, nil, ms, key, timestamp, value, expVal, allowIfDoesNotExist, txn) + return mvccConditionalPutUsingIter( + ctx, writer, nil, ms, key, timestamp, localTimestamp, value, expVal, allowIfDoesNotExist, txn) } func mvccConditionalPutUsingIter( @@ -1767,27 +1788,27 @@ func mvccConditionalPutUsingIter( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, expBytes []byte, allowNoExisting CPutMissingBehavior, txn *roachpb.Transaction, ) error { - return mvccPutUsingIter( - ctx, writer, iter, ms, key, timestamp, noValue, txn, - func(existVal optionalValue) ([]byte, error) { - if expValPresent, existValPresent := len(expBytes) != 0, existVal.IsPresent(); expValPresent && existValPresent { - if !bytes.Equal(expBytes, existVal.TagAndDataBytes()) { - return nil, &roachpb.ConditionFailedError{ - ActualValue: existVal.ToPointer(), - } - } - } else if expValPresent != existValPresent && (existValPresent || !bool(allowNoExisting)) { + valueFn := func(existVal optionalValue) ([]byte, error) { + if expValPresent, existValPresent := len(expBytes) != 0, existVal.IsPresent(); expValPresent && existValPresent { + if !bytes.Equal(expBytes, existVal.TagAndDataBytes()) { return nil, &roachpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } - return value.RawBytes, nil - }) + } else if expValPresent != existValPresent && (existValPresent || !bool(allowNoExisting)) { + return nil, &roachpb.ConditionFailedError{ + ActualValue: existVal.ToPointer(), + } + } + return value.RawBytes, nil + } + return mvccPutUsingIter(ctx, writer, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) } // MVCCInitPut sets the value for a specified key if the key doesn't exist. It @@ -1805,13 +1826,14 @@ func MVCCInitPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, failOnTombstones bool, txn *roachpb.Transaction, ) error { iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) defer iter.Close() - return mvccInitPutUsingIter(ctx, rw, iter, ms, key, timestamp, value, failOnTombstones, txn) + return mvccInitPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, value, failOnTombstones, txn) } // MVCCBlindInitPut is a fast-path of MVCCInitPut. See the MVCCInitPut @@ -1828,11 +1850,13 @@ func MVCCBlindInitPut( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, failOnTombstones bool, txn *roachpb.Transaction, ) error { - return mvccInitPutUsingIter(ctx, rw, nil, ms, key, timestamp, value, failOnTombstones, txn) + return mvccInitPutUsingIter( + ctx, rw, nil, ms, key, timestamp, localTimestamp, value, failOnTombstones, txn) } func mvccInitPutUsingIter( @@ -1842,27 +1866,27 @@ func mvccInitPutUsingIter( ms *enginepb.MVCCStats, key roachpb.Key, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, value roachpb.Value, failOnTombstones bool, txn *roachpb.Transaction, ) error { - return mvccPutUsingIter( - ctx, rw, iter, ms, key, timestamp, noValue, txn, - func(existVal optionalValue) ([]byte, error) { - if failOnTombstones && existVal.IsTombstone() { - // We found a tombstone and failOnTombstones is true: fail. - return nil, &roachpb.ConditionFailedError{ - ActualValue: existVal.ToPointer(), - } + valueFn := func(existVal optionalValue) ([]byte, error) { + if failOnTombstones && existVal.IsTombstone() { + // We found a tombstone and failOnTombstones is true: fail. + return nil, &roachpb.ConditionFailedError{ + ActualValue: existVal.ToPointer(), } - if existVal.IsPresent() && !existVal.EqualTagAndData(value) { - // The existing value does not match the supplied value. - return nil, &roachpb.ConditionFailedError{ - ActualValue: existVal.ToPointer(), - } + } + if existVal.IsPresent() && !existVal.EqualTagAndData(value) { + // The existing value does not match the supplied value. + return nil, &roachpb.ConditionFailedError{ + ActualValue: existVal.ToPointer(), } - return value.RawBytes, nil - }) + } + return value.RawBytes, nil + } + return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) } // mvccKeyFormatter is an fmt.Formatter for MVCC Keys. @@ -2163,6 +2187,7 @@ func MVCCDeleteRange( key, endKey roachpb.Key, max int64, timestamp hlc.Timestamp, + localTimestamp hlc.ClockTimestamp, txn *roachpb.Transaction, returnKeys bool, ) ([]roachpb.Key, *roachpb.Span, int64, error) { @@ -2189,7 +2214,9 @@ func MVCCDeleteRange( var keys []roachpb.Key for i, kv := range res.KVs { - if err := mvccPutInternal(ctx, rw, iter, ms, kv.Key, timestamp, nil, txn, buf, nil); err != nil { + if err := mvccPutInternal( + ctx, rw, iter, ms, kv.Key, timestamp, localTimestamp, nil, txn, buf, nil, + ); err != nil { return nil, nil, 0, err } if returnKeys { diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index cd3f571bf4e0..702040da32b2 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -55,13 +55,13 @@ import ( // resolve_intent t= k= [status=] // check_intent k= [none] // -// cput [t=] [ts=[,]] [resolve [status=]] k= v= [raw] [cond=] -// del [t=] [ts=[,]] [resolve [status=]] k= -// del_range [t=] [ts=[,]] [resolve [status=]] k= [end=] [max=] [returnKeys] -// get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] -// increment [t=] [ts=[,]] [resolve [status=]] k= [inc=] -// put [t=] [ts=[,]] [resolve [status=]] k= v= [raw] -// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] +// cput [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] [cond=] +// del [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= +// del_range [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= [end=] [max=] [returnKeys] +// increment [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= [inc=] +// put [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] +// get [t=] [ts=[,]] [resolve [status=]] k= [inconsistent] [tombstones] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] +// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] // // merge [ts=[,]] k= v= [raw] // @@ -423,7 +423,7 @@ func cmdTxnBegin(e *evalCtx) error { var txnName string e.scanArg("t", &txnName) ts := e.getTs(nil) - globalUncertaintyLimit := e.getTsWithName(nil, "globalUncertaintyLimit") + globalUncertaintyLimit := e.getTsWithName("globalUncertaintyLimit") key := roachpb.KeyMin if e.hasArg("k") { key = e.getKey() @@ -587,6 +587,7 @@ func cmdClearRange(e *evalCtx) error { func cmdCPut(e *evalCtx) error { txn := e.getTxn(optional) ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) key := e.getKey() val := e.getVal() @@ -603,7 +604,7 @@ func cmdCPut(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("cput", func(rw ReadWriter) error { - if err := MVCCConditionalPut(e.ctx, rw, nil, key, ts, val, expVal, behavior, txn); err != nil { + if err := MVCCConditionalPut(e.ctx, rw, nil, key, ts, localTs, val, expVal, behavior, txn); err != nil { return err } if resolve { @@ -617,9 +618,10 @@ func cmdDelete(e *evalCtx) error { txn := e.getTxn(optional) key := e.getKey() ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) resolve, resolveStatus := e.getResolve() return e.withWriter("del", func(rw ReadWriter) error { - if err := MVCCDelete(e.ctx, rw, nil, key, ts, txn); err != nil { + if err := MVCCDelete(e.ctx, rw, nil, key, ts, localTs, txn); err != nil { return err } if resolve { @@ -633,6 +635,7 @@ func cmdDeleteRange(e *evalCtx) error { txn := e.getTxn(optional) key, endKey := e.getKeyRange() ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) returnKeys := e.hasArg("returnKeys") max := 0 if e.hasArg("max") { @@ -641,7 +644,8 @@ func cmdDeleteRange(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("del_range", func(rw ReadWriter) error { - deleted, resumeSpan, num, err := MVCCDeleteRange(e.ctx, rw, nil, key, endKey, int64(max), ts, txn, returnKeys) + deleted, resumeSpan, num, err := MVCCDeleteRange( + e.ctx, rw, nil, key, endKey, int64(max), ts, localTs, txn, returnKeys) if err != nil { return err } @@ -676,8 +680,8 @@ func cmdGet(e *evalCtx) error { opts.FailOnMoreRecent = true } opts.Uncertainty = uncertainty.Interval{ - GlobalLimit: e.getTsWithName(nil, "globalUncertaintyLimit"), - LocalLimit: hlc.ClockTimestamp(e.getTsWithName(nil, "localUncertaintyLimit")), + GlobalLimit: e.getTsWithName("globalUncertaintyLimit"), + LocalLimit: hlc.ClockTimestamp(e.getTsWithName("localUncertaintyLimit")), } if opts.Txn != nil { if !opts.Uncertainty.GlobalLimit.IsEmpty() { @@ -703,6 +707,7 @@ func cmdGet(e *evalCtx) error { func cmdIncrement(e *evalCtx) error { txn := e.getTxn(optional) ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) key := e.getKey() inc := int64(1) @@ -715,7 +720,7 @@ func cmdIncrement(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("increment", func(rw ReadWriter) error { - curVal, err := MVCCIncrement(e.ctx, rw, nil, key, ts, txn, inc) + curVal, err := MVCCIncrement(e.ctx, rw, nil, key, ts, localTs, txn, inc) if err != nil { return err } @@ -746,6 +751,7 @@ func cmdMerge(e *evalCtx) error { func cmdPut(e *evalCtx) error { txn := e.getTxn(optional) ts := e.getTs(txn) + localTs := hlc.ClockTimestamp(e.getTsWithName("localTs")) key := e.getKey() val := e.getVal() @@ -753,7 +759,7 @@ func cmdPut(e *evalCtx) error { resolve, resolveStatus := e.getResolve() return e.withWriter("put", func(rw ReadWriter) error { - if err := MVCCPut(e.ctx, rw, nil, key, ts, val, txn); err != nil { + if err := MVCCPut(e.ctx, rw, nil, key, ts, localTs, val, txn); err != nil { return err } if resolve { @@ -782,8 +788,8 @@ func cmdScan(e *evalCtx) error { opts.FailOnMoreRecent = true } opts.Uncertainty = uncertainty.Interval{ - GlobalLimit: e.getTsWithName(nil, "globalUncertaintyLimit"), - LocalLimit: hlc.ClockTimestamp(e.getTsWithName(nil, "localUncertaintyLimit")), + GlobalLimit: e.getTsWithName("globalUncertaintyLimit"), + LocalLimit: hlc.ClockTimestamp(e.getTsWithName("localUncertaintyLimit")), } if opts.Txn != nil { if !opts.Uncertainty.GlobalLimit.IsEmpty() { @@ -899,10 +905,14 @@ func (e *evalCtx) getResolve() (bool, roachpb.TransactionStatus) { } func (e *evalCtx) getTs(txn *roachpb.Transaction) hlc.Timestamp { - return e.getTsWithName(txn, "ts") + return e.getTsWithTxnAndName(txn, "ts") } -func (e *evalCtx) getTsWithName(txn *roachpb.Transaction, name string) hlc.Timestamp { +func (e *evalCtx) getTsWithName(name string) hlc.Timestamp { + return e.getTsWithTxnAndName(nil, name) +} + +func (e *evalCtx) getTsWithTxnAndName(txn *roachpb.Transaction, name string) hlc.Timestamp { var ts hlc.Timestamp if txn != nil { ts = txn.ReadTimestamp diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index e5f971b07c2a..28a2bf88aec6 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -538,7 +538,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } @@ -576,7 +576,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { }) // Exercise deletion. - if err := MVCCDelete(ctx, e, nil, testKey1, ts3, nil); err != nil { + if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. @@ -597,7 +597,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { ReadTimestamp: ts4, } txn1Val := roachpb.Value{RawBytes: testValue4} - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { t.Fatal(err) } @@ -675,7 +675,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } @@ -710,7 +710,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { }) // Exercise deletion. - if err := MVCCDelete(ctx, e, nil, testKey1, ts3, nil); err != nil { + if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. @@ -731,7 +731,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { ReadTimestamp: ts4, } txn1Val := roachpb.Value{RawBytes: testValue4} - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { t.Fatal(err) } @@ -797,7 +797,7 @@ func TestMVCCIncrementalIteratorInlinePolicy(t *testing.T) { defer e.Close() for _, kv := range []MVCCKeyValue{inline1_1_1, kv2_1_1, kv2_2_2, inline3_2_1} { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } @@ -912,11 +912,11 @@ func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { defer e.Close() for _, kv := range []MVCCKeyValue{kv1_1_1, kv1_2_2, kv1_3_3, kv2_1_1} { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } - if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, val, &txn); err != nil { + if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val, &txn); err != nil { t.Fatal(err) } t.Run(engineImpl.name, func(t *testing.T) { @@ -1096,7 +1096,7 @@ func TestMVCCIncrementalIterator(t *testing.T) { for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } @@ -1114,18 +1114,18 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("kv [1-2)", assertEqualKVs(e, testKey1, testKey2, tsMin, tsMax, latest, kvs(kv1_2_2))) // Exercise deletion. - if err := MVCCDelete(ctx, e, nil, testKey1, ts3, nil); err != nil { + if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, latest, kvs(kv1Deleted3, kv2_2_2))) // Exercise intent handling. txn1, txn1Val, intentErr1 := makeKVTxn(testKey1, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { t.Fatal(err) } txn2, txn2Val, intentErr2 := makeKVTxn(testKey2, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, txn2Val, &txn2); err != nil { + if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, txn2Val, &txn2); err != nil { t.Fatal(err) } t.Run("intents-1", @@ -1164,7 +1164,7 @@ func TestMVCCIncrementalIterator(t *testing.T) { for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, v, nil); err != nil { + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } } @@ -1182,18 +1182,18 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("kv [1-2)", assertEqualKVs(e, testKey1, testKey2, tsMin, tsMax, all, kvs(kv1_2_2, kv1_1_1))) // Exercise deletion. - if err := MVCCDelete(ctx, e, nil, testKey1, ts3, nil); err != nil { + if err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, all, kvs(kv1Deleted3, kv1_2_2, kv2_2_2))) // Exercise intent handling. txn1, txn1Val, intentErr1 := makeKVTxn(testKey1, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { t.Fatal(err) } txn2, txn2Val, intentErr2 := makeKVTxn(testKey2, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, txn2Val, &txn2); err != nil { + if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, txn2Val, &txn2); err != nil { t.Fatal(err) } // Single intent tests are verifying behavior when intent collection is not enabled. @@ -1280,7 +1280,7 @@ func TestMVCCIncrementalIteratorIntentRewrittenConcurrently(t *testing.T) { }, ReadTimestamp: ts1, } - if err := MVCCPut(ctx, e, nil, kA, ts1, vA1, txn); err != nil { + if err := MVCCPut(ctx, e, nil, kA, ts1, hlc.ClockTimestamp{}, vA1, txn); err != nil { t.Fatal(err) } @@ -1298,7 +1298,7 @@ func TestMVCCIncrementalIteratorIntentRewrittenConcurrently(t *testing.T) { // in intentInterleavingIter to be violated. b := e.NewBatch() defer b.Close() - if err := MVCCPut(ctx, b, nil, kA, ts1, vA2, txn); err != nil { + if err := MVCCPut(ctx, b, nil, kA, ts1, hlc.ClockTimestamp{}, vA2, txn); err != nil { return err } return b.Commit(false) @@ -1394,17 +1394,17 @@ func TestMVCCIncrementalIteratorIntentDeletion(t *testing.T) { // kA:3 -> vA3 // kA:2 -> vA2 // kB -> (intent deletion) - require.NoError(t, MVCCPut(ctx, db, nil, kA, txnA1.ReadTimestamp, vA1, txnA1)) - require.NoError(t, MVCCPut(ctx, db, nil, kB, txnB1.ReadTimestamp, vB1, txnB1)) - require.NoError(t, MVCCPut(ctx, db, nil, kC, txnC1.ReadTimestamp, vC1, txnC1)) + require.NoError(t, MVCCPut(ctx, db, nil, kA, txnA1.ReadTimestamp, hlc.ClockTimestamp{}, vA1, txnA1)) + require.NoError(t, MVCCPut(ctx, db, nil, kB, txnB1.ReadTimestamp, hlc.ClockTimestamp{}, vB1, txnB1)) + require.NoError(t, MVCCPut(ctx, db, nil, kC, txnC1.ReadTimestamp, hlc.ClockTimestamp{}, vC1, txnC1)) require.NoError(t, db.Flush()) require.NoError(t, db.Compact()) _, err := MVCCResolveWriteIntent(ctx, db, nil, intent(txnA1)) require.NoError(t, err) _, err = MVCCResolveWriteIntent(ctx, db, nil, intent(txnB1)) require.NoError(t, err) - require.NoError(t, MVCCPut(ctx, db, nil, kA, ts2, vA2, nil)) - require.NoError(t, MVCCPut(ctx, db, nil, kA, txnA3.WriteTimestamp, vA3, txnA3)) + require.NoError(t, MVCCPut(ctx, db, nil, kA, ts2, hlc.ClockTimestamp{}, vA2, nil)) + require.NoError(t, MVCCPut(ctx, db, nil, kA, txnA3.WriteTimestamp, hlc.ClockTimestamp{}, vA3, txnA3)) require.NoError(t, db.Flush()) // The kA ts1 intent has been resolved. There's now a new intent on kA, but @@ -1453,9 +1453,7 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { put := func(key, value string, ts int64, txn *roachpb.Transaction) { v := roachpb.MakeValueFromString(value) - if err := MVCCPut( - ctx, db1, nil, roachpb.Key(key), hlc.Timestamp{WallTime: ts}, v, txn, - ); err != nil { + if err := MVCCPut(ctx, db1, nil, roachpb.Key(key), hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, v, txn); err != nil { t.Fatal(err) } } @@ -1760,8 +1758,7 @@ func BenchmarkMVCCIncrementalIteratorForOldData(b *testing.B) { value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, valueSize)) value.InitChecksum(key) ts := hlc.Timestamp{WallTime: baseTimestamp + 100*int64(i%keyAgeInterval)} - if err := MVCCPut( - context.Background(), batch, nil /* ms */, key, ts, value, nil); err != nil { + if err := MVCCPut(context.Background(), batch, nil, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatal(err) } } diff --git a/pkg/storage/mvcc_logical_ops_test.go b/pkg/storage/mvcc_logical_ops_test.go index bd690ba791be..bf14945e23d5 100644 --- a/pkg/storage/mvcc_logical_ops_test.go +++ b/pkg/storage/mvcc_logical_ops_test.go @@ -39,36 +39,36 @@ func TestMVCCOpLogWriter(t *testing.T) { defer ol.Close() // Write a value and an intent. - if err := MVCCPut(ctx, ol, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{Logical: 2}) - if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } // Write a value and an intent on local keys. localKey := keys.MakeRangeIDPrefix(1) - if err := MVCCPut(ctx, ol, nil, localKey, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, ol, nil, localKey, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } // Update the intents and write another. txn1ts.Sequence++ txn1ts.WriteTimestamp = hlc.Timestamp{Logical: 3} - if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } // Set the txn timestamp to a larger value than the intent. txn1LargerTS := makeTxn(*txn1, hlc.Timestamp{Logical: 4}) txn1LargerTS.WriteTimestamp = hlc.Timestamp{Logical: 4} - if err := MVCCPut(ctx, ol, nil, testKey2, txn1LargerTS.ReadTimestamp, value3, txn1LargerTS); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey2, txn1LargerTS.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1LargerTS); err != nil { t.Fatal(err) } @@ -92,7 +92,7 @@ func TestMVCCOpLogWriter(t *testing.T) { // Write another intent, push it, then abort it. txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 5}) - if err := MVCCPut(ctx, ol, nil, testKey3, txn2ts.ReadTimestamp, value4, txn2ts); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey3, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value4, txn2ts); err != nil { t.Fatal(err) } txn2Pushed := *txn2 diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 5cc09743c86d..855d5e9fe01d 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -97,7 +97,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { ts1 := hlc.Timestamp{WallTime: 1e9} // Put a value. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, value, nil); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } @@ -122,7 +122,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts3}, ReadTimestamp: ts3, } - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, txn); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -180,7 +180,7 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { } // Write an intent at t=1s. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -262,7 +262,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { } // Write an intent. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -352,7 +352,7 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { // Write an intent. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -417,7 +417,7 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { }).Size()) require.EqualValues(t, m2ValSize, 64) - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, txn); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -470,7 +470,7 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { } // Write a deletion tombstone intent. - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, txn); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -528,7 +528,7 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { }).Size()) require.EqualValues(t, m2ValSize, 54) - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -580,7 +580,7 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { ts3 := hlc.Timestamp{WallTime: 3e9} // Write a non-transactional tombstone at t=1s. - if err := MVCCDelete(ctx, engine, aggMS, key, ts1, nil /* txn */); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -604,7 +604,7 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, ReadTimestamp: ts2, } - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, txn); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -724,7 +724,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { // Write a non-transactional value at t=1s. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, value, nil /* txn */); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, nil); err != nil { t.Fatal(err) } @@ -754,7 +754,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, ReadTimestamp: ts2, } - if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, txn); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -835,7 +835,7 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { require.EqualValues(t, vVal2Size, 14) txn.WriteTimestamp.Forward(ts3) - if err := MVCCPut(ctx, engine, &aggMS, key, txn.ReadTimestamp, val2, txn); err != nil { + if err := MVCCPut(ctx, engine, &aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { t.Fatal(err) } @@ -894,10 +894,10 @@ func TestMVCCStatsDelDelGC(t *testing.T) { ts2 := hlc.Timestamp{WallTime: 2e9} // Write tombstones at ts1 and ts2. - if err := MVCCDelete(ctx, engine, aggMS, key, ts1, nil); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } - if err := MVCCDelete(ctx, engine, aggMS, key, ts2, nil); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -975,7 +975,7 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { } // Write an intent at 2s+1. value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -1020,7 +1020,7 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { {Sequence: 0, Value: value.RawBytes}, }, }).Size()) - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, value, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) } @@ -1068,7 +1068,7 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { // Write a value at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, val1, nil /* txn */); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { t.Fatal(err) } @@ -1094,7 +1094,7 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { // Delete the value at ts5. - if err := MVCCDelete(ctx, engine, aggMS, key, ts2, nil /* txn */); err != nil { + if err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -1161,7 +1161,7 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { // Write an intent at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, val1, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val1, txn); err != nil { t.Fatal(err) } @@ -1210,7 +1210,7 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { }).Size()) require.EqualValues(t, mVal2Size, 64) - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, val2, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { t.Fatal(err) } @@ -1251,7 +1251,7 @@ func TestMVCCStatsTxnSysPutAbort(t *testing.T) { // Write a system intent at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, val1, txn); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val1, txn); err != nil { t.Fatal(err) } @@ -1321,7 +1321,7 @@ func TestMVCCStatsSysPutPut(t *testing.T) { // Write a value at ts1. val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, val1, nil /* txn */); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { t.Fatal(err) } @@ -1347,7 +1347,7 @@ func TestMVCCStatsSysPutPut(t *testing.T) { // Put another value at ts2. - if err := MVCCPut(ctx, engine, aggMS, key, ts2, val2, nil /* txn */); err != nil { + if err := MVCCPut(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, val2, nil); err != nil { t.Fatal(err) } @@ -1495,7 +1495,7 @@ func TestMVCCStatsRandomized(t *testing.T) { actions := make(map[string]func(*state) string) actions["Put"] = func(s *state) string { - if err := MVCCPut(ctx, s.eng, s.MS, s.key, s.TS, s.rngVal(), s.Txn); err != nil { + if err := MVCCPut(ctx, s.eng, s.MS, s.key, s.TS, hlc.ClockTimestamp{}, s.rngVal(), s.Txn); err != nil { return err.Error() } return "" @@ -1503,13 +1503,13 @@ func TestMVCCStatsRandomized(t *testing.T) { actions["InitPut"] = func(s *state) string { failOnTombstones := (s.rng.Intn(2) == 0) desc := fmt.Sprintf("failOnTombstones=%t", failOnTombstones) - if err := MVCCInitPut(ctx, s.eng, s.MS, s.key, s.TS, s.rngVal(), failOnTombstones, s.Txn); err != nil { + if err := MVCCInitPut(ctx, s.eng, s.MS, s.key, s.TS, hlc.ClockTimestamp{}, s.rngVal(), failOnTombstones, s.Txn); err != nil { return desc + ": " + err.Error() } return desc } actions["Del"] = func(s *state) string { - if err := MVCCDelete(ctx, s.eng, s.MS, s.key, s.TS, s.Txn); err != nil { + if err := MVCCDelete(ctx, s.eng, s.MS, s.key, s.TS, hlc.ClockTimestamp{}, s.Txn); err != nil { return err.Error() } return "" @@ -1525,7 +1525,9 @@ func TestMVCCStatsRandomized(t *testing.T) { } else { keyMin = keys.LocalMax } - if _, _, _, err := MVCCDeleteRange(ctx, s.eng, s.MS, keyMin, keyMax, max, s.TS, s.Txn, returnKeys); err != nil { + if _, _, _, err := MVCCDeleteRange( + ctx, s.eng, s.MS, keyMin, keyMax, max, s.TS, hlc.ClockTimestamp{}, s.Txn, returnKeys, + ); err != nil { return desc + ": " + err.Error() } return desc diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 539d2a0e6f33..6060927203a2 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -293,10 +293,10 @@ func TestMVCCGetNoMoreOldVersion(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } @@ -322,7 +322,7 @@ func TestMVCCGetAndDelete(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) @@ -333,7 +333,7 @@ func TestMVCCGetAndDelete(t *testing.T) { t.Fatal("the value should not be empty") } - err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, nil) + err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil) if err != nil { t.Fatal(err) } @@ -382,15 +382,11 @@ func TestMVCCWriteWithOlderTimestampAfterDeletionOfNonexistentKey(t *testing.T) engine := engineImpl.create() defer engine.Close() - if err := MVCCDelete( - context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, nil, - ); err != nil { + if err := MVCCDelete(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } - if err := MVCCPut( - context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil, - ); !testutils.IsError( + if err := MVCCPut(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); !testutils.IsError( err, "write for key \"/db1\" at timestamp 0.000000001,0 too old; wrote at 0.000000003,1", ) { t.Fatal(err) @@ -437,7 +433,7 @@ func TestMVCCInlineWithTxn(t *testing.T) { defer engine.Close() // Put an inline value. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } @@ -459,7 +455,7 @@ func TestMVCCInlineWithTxn(t *testing.T) { } // Verify inline put with txn is an error. - err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{}, value2, txn2) + err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{}, hlc.ClockTimestamp{}, value2, txn2) if !testutils.IsError(err, "writes not allowed within transactions") { t.Errorf("unexpected error: %+v", err) } @@ -477,7 +473,7 @@ func TestMVCCDeleteMissingKey(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, nil); err != nil { + if err := MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } // Verify nothing is written to the engine. @@ -501,7 +497,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn); err != nil { t.Fatal(err) } @@ -515,7 +511,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { txn.Sequence++ txn.WriteTimestamp = hlc.Timestamp{WallTime: 3} - if err := MVCCDelete(ctx, engine, nil, testKey1, txn.ReadTimestamp, txn); err != nil { + if err := MVCCDelete(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } @@ -559,7 +555,7 @@ func TestMVCCGetWriteIntentError(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } @@ -616,7 +612,7 @@ func TestMVCCScanWriteIntentError(t *testing.T) { for i, kv := range fixtureKVs { v := *protoutil.Clone(&kv.Value).(*roachpb.Value) v.Timestamp = hlc.Timestamp{} - if err := MVCCPut(ctx, engine, nil, kv.Key, kv.Value.Timestamp, v, txnMap[i]); err != nil { + if err := MVCCPut(ctx, engine, nil, kv.Key, kv.Value.Timestamp, hlc.ClockTimestamp{}, v, txnMap[i]); err != nil { t.Fatal(err) } } @@ -723,11 +719,11 @@ func TestMVCCGetInconsistent(t *testing.T) { defer engine.Close() // Put two values to key 1, the latest with a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } @@ -757,7 +753,7 @@ func TestMVCCGetInconsistent(t *testing.T) { } // Write a single intent for key 2 and verify get returns empty. - if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, value1, txn2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn2); err != nil { t.Fatal(err) } val, intent, err := MVCCGet(ctx, engine, testKey2, hlc.Timestamp{WallTime: 2}, @@ -797,11 +793,11 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { v2 := roachpb.MakeValueFromBytes(bytes2) // Put two values to key 1, the latest with a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, v1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, v1, nil); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, v2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, v2, txn1ts); err != nil { t.Fatal(err) } @@ -843,7 +839,7 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { { // Write a single intent for key 2 and verify get returns empty. - if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, v1, txn2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, hlc.ClockTimestamp{}, v1, txn2); err != nil { t.Fatal(err) } val := roachpb.Value{} @@ -862,10 +858,10 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { // Write a malformed value (not an encoded MVCCKeyValue) and a // write intent to key 3; the parse error is returned instead of the // write intent. - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, txn1ts.ReadTimestamp, v2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, v2, txn1ts); err != nil { t.Fatal(err) } val := roachpb.Value{} @@ -903,7 +899,7 @@ func TestMVCCInvalidateIterator(t *testing.T) { ts2 := hlc.Timestamp{WallTime: 2} key := roachpb.Key("a") - if err := MVCCPut(ctx, engine, nil, key, ts1, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, key, ts1, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } @@ -1005,7 +1001,7 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { iter.Next() // key2/5 // Lay down an intent on key3, which will go at key3/0 and sort before key3/5. - err = MVCCDelete(context.Background(), batch, nil, testKey3, txn.WriteTimestamp, txn) + err = MVCCDelete(context.Background(), batch, nil, testKey3, txn.WriteTimestamp, hlc.ClockTimestamp{}, txn) if err != nil { t.Fatal(err) } @@ -1022,28 +1018,28 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { } func mvccScanTest(ctx context.Context, t *testing.T, engine Engine) { - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 4}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 4}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 5}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } @@ -1143,19 +1139,19 @@ func TestMVCCScanMaxNum(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } @@ -1235,19 +1231,19 @@ func TestMVCCScanWithKeyPrefix(t *testing.T) { // b // In this case, if we scan from "a"-"b", we wish to skip // a and a and find "aa'. - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 2}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 2}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/b"), hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/b"), hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } @@ -1277,17 +1273,17 @@ func TestMVCCScanInTxn(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, value3, txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } @@ -1339,24 +1335,24 @@ func TestMVCCScanInconsistent(t *testing.T) { ts4 := hlc.Timestamp{WallTime: 4} ts5 := hlc.Timestamp{WallTime: 5} ts6 := hlc.Timestamp{WallTime: 6} - if err := MVCCPut(ctx, engine, nil, testKey1, ts1, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, ts1, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } txn1ts2 := makeTxn(*txn1, ts2) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts2.ReadTimestamp, value2, txn1ts2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts2); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, ts3, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, ts3, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, ts4, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, ts4, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } txn2ts5 := makeTxn(*txn2, ts5) - if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts5.ReadTimestamp, value3, txn2ts5); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts5.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2ts5); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, ts6, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, ts6, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } @@ -1417,29 +1413,28 @@ func TestMVCCDeleteRange(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value5, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { t.Fatal(err) } // Attempt to delete two keys. - deleted, resumeSpan, num, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, 2, hlc.Timestamp{WallTime: 2}, nil, false, - ) + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) if err != nil { t.Fatal(err) } @@ -1494,8 +1489,8 @@ func TestMVCCDeleteRange(t *testing.T) { } // Attempt to delete no keys. - deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, -1, hlc.Timestamp{WallTime: 2}, nil, false) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + -1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) if err != nil { t.Fatal(err) } @@ -1522,8 +1517,8 @@ func TestMVCCDeleteRange(t *testing.T) { t.Fatal("the value should not be empty") } - deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, testKey4, keyMax, 0, hlc.Timestamp{WallTime: 2}, nil, false) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey4, keyMax, + 0, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) if err != nil { t.Fatal(err) } @@ -1547,8 +1542,8 @@ func TestMVCCDeleteRange(t *testing.T) { t.Fatalf("the value should not be empty: %+v", res.KVs) } - deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, localMax, testKey2, 0, hlc.Timestamp{WallTime: 2}, nil, false) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, localMax, testKey2, + 0, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) if err != nil { t.Fatal(err) } @@ -1583,28 +1578,28 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value5, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { t.Fatal(err) } // Attempt to delete two keys. - deleted, resumeSpan, num, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, 2, hlc.Timestamp{WallTime: 2}, nil, true) + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) if err != nil { t.Fatal(err) } @@ -1638,8 +1633,8 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { } // Attempt to delete no keys. - deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, -1, hlc.Timestamp{WallTime: 2}, nil, true) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + -1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) if err != nil { t.Fatal(err) } @@ -1666,8 +1661,8 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { t.Fatal("the value should not be empty") } - deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, testKey4, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, nil, true) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey4, keyMax, + math.MaxInt64, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) if err != nil { t.Fatal(err) } @@ -1697,8 +1692,8 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { t.Fatal("the value should not be empty") } - deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, localMax, testKey2, math.MaxInt64, hlc.Timestamp{WallTime: 2}, nil, true) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, localMax, testKey2, + math.MaxInt64, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) if err != nil { t.Fatal(err) } @@ -1734,31 +1729,29 @@ func TestMVCCDeleteRangeFailed(t *testing.T) { defer engine.Close() txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey2, txn.ReadTimestamp, value2, txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn); err != nil { t.Fatal(err) } txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, value3, txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, nil, false, - ); err == nil { + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil, false); err == nil { t.Fatal("expected error on uncommitted write intent") } txn.Sequence++ - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey4, math.MaxInt64, txn.ReadTimestamp, txn, false, - ); err != nil { + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, + math.MaxInt64, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, false); err != nil { t.Fatal(err) } }) @@ -1778,21 +1771,21 @@ func TestMVCCDeleteRangeConcurrentTxn(t *testing.T) { txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn2ts := makeTxn(*txn2, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts.ReadTimestamp, value3, txn2ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2ts); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey4, math.MaxInt64, txn1ts.ReadTimestamp, txn1ts, false, + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, + math.MaxInt64, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, txn1ts, false, ); err == nil { t.Fatal("expected error on uncommitted write intent") } @@ -1812,31 +1805,23 @@ func TestMVCCUncommittedDeleteRangeVisible(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut( - ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil, - ); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut( - ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil, - ); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut( - ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil, - ); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCDelete( - ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, nil, - ); err != nil { + if err := MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 2, Logical: 2}) - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey1, testKey4, math.MaxInt64, txn.ReadTimestamp, txn, false, + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey1, testKey4, + math.MaxInt64, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, false, ); err != nil { t.Fatal(err) } @@ -1862,15 +1847,15 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { t.Run(engineImpl.name, func(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) if err != nil { t.Fatal(err) } - err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, value2, nil) + err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) if err != nil { t.Fatal(err) } - err = MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 5}, nil) + err = MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil) if err != nil { t.Fatal(err) } @@ -1878,9 +1863,8 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { // Delete at a time before the tombstone. Should return a WriteTooOld error. b := engine.NewBatch() defer b.Close() - keys, resume, keyCount, err := MVCCDeleteRange( - ctx, b, nil, testKey1, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 4}, nil, true, - ) + keys, resume, keyCount, err := MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 4}, hlc.ClockTimestamp{}, nil, true) require.Nil(t, keys) require.Nil(t, resume) require.Equal(t, int64(0), keyCount) @@ -1890,9 +1874,8 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { // Delete at the same time as the tombstone. Should return a WriteTooOld error. b = engine.NewBatch() defer b.Close() - keys, resume, keyCount, err = MVCCDeleteRange( - ctx, b, nil, testKey1, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 5}, nil, true, - ) + keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil, true) require.Nil(t, keys) require.Nil(t, resume) require.Equal(t, int64(0), keyCount) @@ -1903,9 +1886,8 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { // include the tombstone in the returned keys. b = engine.NewBatch() defer b.Close() - keys, resume, keyCount, err = MVCCDeleteRange( - ctx, b, nil, testKey1, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 6}, nil, true, - ) + keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 6}, hlc.ClockTimestamp{}, nil, true) require.Equal(t, []roachpb.Key{testKey1}, keys) require.Nil(t, resume) require.Equal(t, int64(1), keyCount) @@ -1935,20 +1917,19 @@ func TestMVCCDeleteRangeInline(t *testing.T) { {testKey4, value4}, {testKey5, value5}, } { - if err := MVCCPut(ctx, engine, nil, kv.key, hlc.Timestamp{Logical: 0}, kv.value, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, kv.key, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, kv.value, nil); err != nil { t.Fatalf("%d: %+v", i, err) } } // Create one non-inline value (non-zero timestamp). - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { t.Fatal(err) } // Attempt to delete two inline keys, should succeed. - deleted, resumeSpan, num, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, 2, hlc.Timestamp{Logical: 0}, nil, true, - ) + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, nil, true) if err != nil { t.Fatal(err) } @@ -1964,23 +1945,23 @@ func TestMVCCDeleteRangeInline(t *testing.T) { // Attempt to delete inline keys at a timestamp; should fail. const inlineMismatchErrString = "put is inline" - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey1, testKey6, 1, hlc.Timestamp{WallTime: 2}, nil, true, + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey1, testKey6, + 1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true, ); !testutils.IsError(err, inlineMismatchErrString) { t.Fatalf("got error %v, expected error with text '%s'", err, inlineMismatchErrString) } // Attempt to delete non-inline key at zero timestamp; should fail. const writeTooOldErrString = "WriteTooOldError" - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey6, keyMax, 1, hlc.Timestamp{Logical: 0}, nil, true, + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey6, keyMax, + 1, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, nil, true, ); !testutils.IsError(err, writeTooOldErrString) { t.Fatalf("got error %v, expected error with text '%s'", err, writeTooOldErrString) } // Attempt to delete inline keys in a transaction; should fail. - if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, 2, hlc.Timestamp{Logical: 0}, txn1, true, + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, txn1, true, ); !testutils.IsError(err, "writes not allowed within transactions") { t.Errorf("unexpected error: %+v", err) } @@ -2076,12 +2057,12 @@ func TestMVCCClearTimeRange(t *testing.T) { // when read. setupKVs := func(t *testing.T) Engine { engine := engineImpl.create() - require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts1, value1, nil)) - require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts2, value2, nil)) - require.NoError(t, MVCCPut(ctx, engine, nil, testKey5, ts2, value2, nil)) - require.NoError(t, MVCCPut(ctx, engine, nil, testKey1, ts3, value3, nil)) - require.NoError(t, MVCCPut(ctx, engine, nil, testKey5, ts4, value4, nil)) - require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts4, value4, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts1, hlc.ClockTimestamp{}, value1, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts2, hlc.ClockTimestamp{}, value2, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey5, ts2, hlc.ClockTimestamp{}, value2, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey1, ts3, hlc.ClockTimestamp{}, value3, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey5, ts4, hlc.ClockTimestamp{}, value4, nil)) + require.NoError(t, MVCCPut(ctx, engine, nil, testKey2, ts4, hlc.ClockTimestamp{}, value4, nil)) return engine } @@ -2241,7 +2222,7 @@ func TestMVCCClearTimeRange(t *testing.T) { txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, ts3, 1, 1) setupKVsWithIntent := func(t *testing.T) Engine { e := setupKVs(t) - require.NoError(t, MVCCPut(ctx, e, &enginepb.MVCCStats{}, testKey3, ts3, value3, &txn)) + require.NoError(t, MVCCPut(ctx, e, &enginepb.MVCCStats{}, testKey3, ts3, hlc.ClockTimestamp{}, value3, &txn)) return e } t.Run("clear everything hitting intent fails", func(t *testing.T) { @@ -2357,24 +2338,24 @@ func TestMVCCClearTimeRangeOnRandomData(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%05d", k)) if rand.Float64() > 0.8 { - require.NoError(t, MVCCDelete(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, nil)) + require.NoError(t, MVCCDelete(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, nil)) } else { v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) - require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, hlc.ClockTimestamp{}, v, nil)) } } swathTime := rand.Intn(randTimeRange-100) + 100 for i := swathStart; i < swathEnd; i++ { key := roachpb.Key(fmt.Sprintf("%05d", i)) v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) - require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: int64(swathTime)}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: int64(swathTime)}, hlc.ClockTimestamp{}, v, nil)) } // Add another swath of keys above to exercise an after-iteration range flush. for i := keyRange; i < keyRange+200; i++ { key := roachpb.Key(fmt.Sprintf("%05d", i)) v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) - require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: int64(randTimeRange + 1)}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: int64(randTimeRange + 1)}, hlc.ClockTimestamp{}, v, nil)) } ms.AgeTo(2000) @@ -2441,25 +2422,25 @@ func TestMVCCInitPut(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, false, nil) + err := MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, false, nil) if err != nil { t.Fatal(err) } // A repeat of the command will still succeed - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 2}, value1, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 2}, hlc.ClockTimestamp{}, value1, false, nil) if err != nil { t.Fatal(err) } // Delete. - err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 3}, nil) + err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, nil) if err != nil { t.Fatal(err) } // Reinserting the value fails if we fail on tombstones. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 4}, value1, true, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 4}, hlc.ClockTimestamp{}, value1, true, nil) if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, nil) { t.Fatalf("the value %s in get result is not a tombstone", e.ActualValue.RawBytes) @@ -2471,13 +2452,13 @@ func TestMVCCInitPut(t *testing.T) { } // But doesn't if we *don't* fail on tombstones. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 5}, value1, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 5}, hlc.ClockTimestamp{}, value1, false, nil) if err != nil { t.Fatal(err) } // A repeat of the command with a different value will fail. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 6}, value2, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 6}, hlc.ClockTimestamp{}, value2, false, nil) if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, value1.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", @@ -2537,14 +2518,14 @@ func TestMVCCInitPutWithTxn(t *testing.T) { txn := *txn1 txn.Sequence++ - err := MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, false, &txn) + err := MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, false, &txn) if err != nil { t.Fatal(err) } // A repeat of the command will still succeed. txn.Sequence++ - err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, false, &txn) + err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, false, &txn) if err != nil { t.Fatal(err) } @@ -2553,7 +2534,7 @@ func TestMVCCInitPutWithTxn(t *testing.T) { // will still succeed. txn.Sequence++ txn.Epoch = 2 - err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, false, &txn) + err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, false, &txn) if err != nil { t.Fatal(err) } @@ -2568,7 +2549,7 @@ func TestMVCCInitPutWithTxn(t *testing.T) { } // Write value4 with an old timestamp without txn...should get an error. - err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), value4, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), hlc.ClockTimestamp{}, value4, false, nil) if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, value2.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", @@ -2593,28 +2574,28 @@ func TestMVCCReverseScan(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 3}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value5, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 3}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value6, nil); err != nil { t.Fatal(err) } @@ -2729,10 +2710,10 @@ func TestMVCCReverseScanFirstKeyInFuture(t *testing.T) { // Before fixing #17825, the MVCC version scan on key3 would fall out of the // scan bounds and if it never found another valid key before reaching // KeyMax, would stop the ReverseScan from continuing. - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } @@ -2769,12 +2750,12 @@ func TestMVCCReverseScanSeeksOverRepeatedKeys(t *testing.T) { // written. Repeat the key enough times to make sure the `SeekForPrev()` // optimization will be used. for i := 1; i <= 10; i++ { - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: int64(i)}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: int64(i)}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 11}) - if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } @@ -2821,7 +2802,7 @@ func TestMVCCReverseScanStopAtSmallestKey(t *testing.T) { defer engine.Close() for i := 1; i <= numPuts; i++ { - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: int64(i)}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: int64(i)}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } } @@ -2862,7 +2843,7 @@ func TestMVCCResolveTxn(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } @@ -2912,12 +2893,12 @@ func TestMVCCResolveNewerIntent(t *testing.T) { defer engine.Close() // Write first value. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1Commit.WriteTimestamp, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1Commit.WriteTimestamp, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } // Now, put down an intent which should return a write too old error // (but will still write the intent at tx1Commit.Timestamp+1. - err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value2, txn1) + err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1) if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { t.Fatalf("expected write too old error; got %s", err) } @@ -2954,7 +2935,7 @@ func TestMVCCResolveIntentTxnTimestampMismatch(t *testing.T) { txn.TxnMeta.WriteTimestamp.Forward(tsEarly.Add(10, 0)) // Write an intent which has txn.WriteTimestamp > meta.timestamp. - if err := MVCCPut(ctx, engine, nil, testKey1, tsEarly, value1, txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, tsEarly, hlc.ClockTimestamp{}, value1, txn); err != nil { t.Fatal(err) } @@ -3005,17 +2986,17 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { t.Run(engineImpl.name, func(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) if err != nil { t.Fatal(err) } - err = MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value2, nil) + err = MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) if err != nil { t.Fatal(err) } // Check nothing is written if the value doesn't match. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value3, value1.TagAndDataBytes(), CPutFailIfMissing, nil) + err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value3, value1.TagAndDataBytes(), CPutFailIfMissing, nil) if err == nil { t.Errorf("unexpected success on conditional put") } @@ -3025,7 +3006,7 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { // But if value does match the most recently written version, we'll get // a write too old error but still write updated value. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value3, value2.TagAndDataBytes(), CPutFailIfMissing, nil) + err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value3, value2.TagAndDataBytes(), CPutFailIfMissing, nil) if err == nil { t.Errorf("unexpected success on conditional put") } @@ -3058,7 +3039,7 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil) if err != nil { t.Fatal(err) } @@ -3067,7 +3048,7 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { // intent is written at the advanced timestamp. txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, txn) + err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { t.Errorf("expected WriteTooOldError on Put; got %v", err) } @@ -3085,7 +3066,7 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { // Put again and verify no WriteTooOldError, but timestamp should continue // to be set to (3,1). txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value3, txn) + err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn) if err != nil { t.Error(err) } @@ -3115,7 +3096,7 @@ func TestMVCCPutNegativeTimestampError(t *testing.T) { timestamp := hlc.Timestamp{WallTime: -1} expectedErrorString := fmt.Sprintf("cannot write to %q at timestamp %s", testKey1, timestamp) - err := MVCCPut(ctx, engine, nil, testKey1, timestamp, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, timestamp, hlc.ClockTimestamp{}, value1, nil) require.EqualError(t, err, expectedErrorString) }) @@ -3138,7 +3119,7 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil) if err != nil { t.Fatal(err) } @@ -3149,7 +3130,7 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) txn.WriteTimestamp = hlc.Timestamp{WallTime: 5} txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, txn) + err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) // Verify that the Put returned a WriteTooOld with the ActualTime set to the // transactions provisional commit timestamp. @@ -3182,7 +3163,7 @@ func TestMVCCAbortTxn(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } @@ -3221,14 +3202,14 @@ func TestMVCCAbortTxnWithPreviousVersion(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value3, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts); err != nil { t.Fatal(err) } @@ -3274,7 +3255,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { // Start with epoch 1. txn := *txn1 txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, &txn); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txn); err != nil { t.Fatal(err) } // Now write with greater timestamp and epoch 2. @@ -3282,22 +3263,22 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { txne2.Sequence++ txne2.Epoch = 2 txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, value2, &txne2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value2, &txne2); err != nil { t.Fatal(err) } // Try a write with an earlier timestamp; this is just ignored. txne2.Sequence++ txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, value1, &txne2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txne2); err != nil { t.Fatal(err) } // Try a write with an earlier epoch; again ignored. - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, &txn); err == nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txn); err == nil { t.Fatal("unexpected success of a write with an earlier epoch") } // Try a write with different value using both later timestamp and epoch. txne2.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, value3, &txne2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value3, &txne2); err != nil { t.Fatal(err) } // Resolve the intent. @@ -3312,7 +3293,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { expTS := txne2Commit.WriteTimestamp.Next() // Now try writing an earlier value without a txn--should get WriteTooOldError. - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value4, nil) + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value4, nil) if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatal("unexpected success") } else if wtoErr.ActualTimestamp != expTS { @@ -3325,7 +3306,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { err, value.Timestamp, expTS, value4.RawBytes, value.RawBytes) } // Now write an intent with exactly the same timestamp--ties also get WriteTooOldError. - err = MVCCPut(ctx, engine, nil, testKey1, txn2.ReadTimestamp, value5, txn2) + err = MVCCPut(ctx, engine, nil, testKey1, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value5, txn2) intentTS := expTS.Next() if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatal("unexpected success") @@ -3373,12 +3354,12 @@ func TestMVCCGetWithDiffEpochs(t *testing.T) { defer engine.Close() // Write initial value without a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } // Now write using txn1, epoch 1. txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { t.Fatal(err) } // Try reading using different txns & epochs. @@ -3434,11 +3415,11 @@ func TestMVCCGetWithDiffEpochsAndTimestamps(t *testing.T) { defer engine.Close() // Write initial value without a txn at timestamp 1. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } // Write another value without a txn at timestamp 3. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } // Now write using txn1, epoch 1. @@ -3446,7 +3427,7 @@ func TestMVCCGetWithDiffEpochsAndTimestamps(t *testing.T) { // Bump epoch 1's write timestamp to timestamp 4. txn1ts.WriteTimestamp = hlc.Timestamp{WallTime: 4} // Expected to hit WriteTooOld error but to still lay down intent. - err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value3, txn1ts) + err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts) if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatalf("unexpectedly not WriteTooOld: %+v", err) } else if expTS, actTS := txn1ts.WriteTimestamp, wtoErr.ActualTimestamp; expTS != actTS { @@ -3503,7 +3484,7 @@ func TestMVCCGetWithOldEpoch(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, value2, txn1e2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1e2); err != nil { t.Fatal(err) } _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ @@ -3548,7 +3529,7 @@ func TestMVCCDeleteRangeWithSequence(t *testing.T) { for i := enginepb.TxnSeq(0); i < 3; i++ { key := append(prefix, []byte(strconv.Itoa(int(i)))...) txn.Sequence = 2 + i - if err := MVCCPut(ctx, engine, nil, key, txn.WriteTimestamp, value1, &txn); err != nil { + if err := MVCCPut(ctx, engine, nil, key, txn.WriteTimestamp, hlc.ClockTimestamp{}, value1, &txn); err != nil { t.Fatal(err) } } @@ -3556,17 +3537,15 @@ func TestMVCCDeleteRangeWithSequence(t *testing.T) { // Perform the initial DeleteRange. const origSeq = 6 txn.Sequence = origSeq - origDeleted, _, origNum, err := MVCCDeleteRange( - ctx, engine, nil, prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, &txn, true, - ) + origDeleted, _, origNum, err := MVCCDeleteRange(ctx, engine, nil, + prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, hlc.ClockTimestamp{}, &txn, true) if err != nil { t.Fatal(err) } txn.Sequence = tc.sequence - deleted, _, num, err := MVCCDeleteRange( - ctx, engine, nil, prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, &txn, true, - ) + deleted, _, num, err := MVCCDeleteRange(ctx, engine, nil, + prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, hlc.ClockTimestamp{}, &txn, true) if tc.expErr != "" && err != nil { if !testutils.IsError(err, tc.expErr) { t.Fatalf("unexpected error: %+v", err) @@ -3609,7 +3588,7 @@ func TestMVCCGetWithPushedTimestamp(t *testing.T) { defer engine.Close() // Start with epoch 1. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } // Resolve the intent, pushing its timestamp forward. @@ -3639,10 +3618,10 @@ func TestMVCCResolveWithDiffEpochs(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, txn1e2.ReadTimestamp, value2, txn1e2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1e2); err != nil { t.Fatal(err) } num, _, err := MVCCResolveWriteIntentRange(ctx, engine, nil, @@ -3685,7 +3664,7 @@ func TestMVCCResolveWithUpdatedTimestamp(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } @@ -3738,7 +3717,7 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ @@ -3800,7 +3779,7 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { } // Add key and resolve despite there being no intent. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { t.Fatal(err) } if _, err := MVCCResolveWriteIntent(ctx, engine, nil, @@ -3809,7 +3788,7 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { } // Write intent and resolve with different txn. - if err := MVCCPut(ctx, engine, nil, testKey2, txn1.ReadTimestamp, value2, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1); err != nil { t.Fatal(err) } @@ -3833,16 +3812,16 @@ func TestMVCCResolveTxnRange(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{Logical: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, txn2.ReadTimestamp, value3, txn2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, txn1.ReadTimestamp, value4, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value4, txn1); err != nil { t.Fatal(err) } @@ -3923,14 +3902,14 @@ func TestMVCCResolveTxnRangeResume(t *testing.T) { key0 := roachpb.Key(fmt.Sprintf("%02d%d", i+0, i+0)) key1 := roachpb.Key(fmt.Sprintf("%02d%d", i+1, i+1)) key2 := roachpb.Key(fmt.Sprintf("%02d%d", i+2, i+2)) - if err := MVCCPut(ctx, engine, nil, key0, txn1.ReadTimestamp, value1, txn1); err != nil { + if err := MVCCPut(ctx, engine, nil, key0, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { t.Fatal(err) } txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 2}) - if err := MVCCPut(ctx, engine, nil, key1, txn2ts.ReadTimestamp, value2, txn2ts); err != nil { + if err := MVCCPut(ctx, engine, nil, key1, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn2ts); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, key2, hlc.Timestamp{Logical: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, key2, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { t.Fatal(err) } } @@ -4049,7 +4028,7 @@ func writeToEngine( log.Infof(ctx, "Put: %s, seq: %d, writets: %s", p.key.String(), txn.Sequence, txn.WriteTimestamp.String()) } - require.NoError(t, MVCCPut(ctx, eng, nil, p.key, txn.ReadTimestamp, p.values[i], txn)) + require.NoError(t, MVCCPut(ctx, eng, nil, p.key, txn.ReadTimestamp, hlc.ClockTimestamp{}, p.values[i], txn)) } } } @@ -4447,7 +4426,7 @@ func TestFindSplitKey(t *testing.T) { v := strings.Repeat("X", 10-len(k)) val := roachpb.MakeValueFromString(v) // Write the key and value through MVCC - if err := MVCCPut(ctx, engine, ms, []byte(k), hlc.Timestamp{Logical: 1}, val, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, []byte(k), hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } } @@ -4752,7 +4731,7 @@ func TestFindValidSplitKeys(t *testing.T) { // between MVCC versions, so this shouldn't have any effect. for j := 1; j <= 3; j++ { ts := hlc.Timestamp{Logical: int32(j)} - if err := MVCCPut(ctx, engine, ms, []byte(k), ts, val, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, []byte(k), ts, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } } @@ -4855,7 +4834,7 @@ func TestFindBalancedSplitKeys(t *testing.T) { expKey = key } val := roachpb.MakeValueFromString(strings.Repeat("X", test.valSizes[j])) - if err := MVCCPut(ctx, engine, ms, key, hlc.Timestamp{Logical: 1}, val, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, key, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } } @@ -4916,14 +4895,14 @@ func TestMVCCGarbageCollect(t *testing.T) { } for _, val := range test.vals[i : i+1] { if i == len(test.vals)-1 && test.isDeleted { - if err := MVCCDelete(ctx, engine, ms, test.key, val.Timestamp, nil); err != nil { + if err := MVCCDelete(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, nil); err != nil { t.Fatal(err) } continue } valCpy := *protoutil.Clone(&val).(*roachpb.Value) valCpy.Timestamp = hlc.Timestamp{} - if err := MVCCPut(ctx, engine, ms, test.key, val.Timestamp, valCpy, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, valCpy, nil); err != nil { t.Fatal(err) } } @@ -5022,7 +5001,7 @@ func TestMVCCGarbageCollectNonDeleted(t *testing.T) { for _, val := range test.vals { valCpy := *protoutil.Clone(&val).(*roachpb.Value) valCpy.Timestamp = hlc.Timestamp{} - if err := MVCCPut(ctx, engine, nil, test.key, val.Timestamp, valCpy, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, test.key, val.Timestamp, hlc.ClockTimestamp{}, valCpy, nil); err != nil { t.Fatal(err) } } @@ -5056,7 +5035,7 @@ func TestMVCCGarbageCollectIntent(t *testing.T) { key := roachpb.Key("a") { val1 := roachpb.MakeValueFromBytes(bytes) - if err := MVCCPut(ctx, engine, nil, key, ts1, val1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { t.Fatal(err) } } @@ -5064,7 +5043,7 @@ func TestMVCCGarbageCollectIntent(t *testing.T) { TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, ReadTimestamp: ts2, } - if err := MVCCDelete(ctx, engine, nil, key, txn.ReadTimestamp, txn); err != nil { + if err := MVCCDelete(ctx, engine, nil, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } keys := []roachpb.GCRequest_GCKey{ @@ -5168,9 +5147,7 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { for _, seconds := range key.timestamps { val := roachpb.MakeValueFromBytes(bytes) ts := toHLC(seconds) - if err := MVCCPut( - ctx, engine, ms, roachpb.Key(key.key), ts, val, nil, - ); err != nil { + if err := MVCCPut(ctx, engine, ms, roachpb.Key(key.key), ts, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } } @@ -5314,7 +5291,7 @@ func TestResolveIntentWithLowerEpoch(t *testing.T) { defer engine.Close() // Lay down an intent with a high epoch. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, value1, txn1e2); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1e2); err != nil { t.Fatal(err) } // Resolve the intent with a low epoch. diff --git a/pkg/storage/pebble_mvcc_scanner_test.go b/pkg/storage/pebble_mvcc_scanner_test.go index 0a61de339f31..597eb9a21c7a 100644 --- a/pkg/storage/pebble_mvcc_scanner_test.go +++ b/pkg/storage/pebble_mvcc_scanner_test.go @@ -211,7 +211,7 @@ func TestMVCCScanWithMemoryAccounting(t *testing.T) { defer batch.Close() for i := 0; i < 10; i++ { key := makeKey(nil, i) - require.NoError(t, MVCCPut(context.Background(), batch, nil, key, ts1, val, &txn1)) + require.NoError(t, MVCCPut(context.Background(), batch, nil, key, ts1, hlc.ClockTimestamp{}, val, &txn1)) } require.NoError(t, batch.Commit(true)) }() diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 44409107043d..c1e95b8a93d7 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -664,7 +664,7 @@ func fillInData(ctx context.Context, engine Engine, data []testValue) error { batch := engine.NewBatch() defer batch.Close() for _, val := range data { - if err := MVCCPut(ctx, batch, nil, val.key, val.timestamp, val.value, val.txn); err != nil { + if err := MVCCPut(ctx, batch, nil, val.key, val.timestamp, hlc.ClockTimestamp{}, val.value, val.txn); err != nil { return err } } diff --git a/pkg/storage/sst_test.go b/pkg/storage/sst_test.go index d05f0f250742..fc467713ce3f 100644 --- a/pkg/storage/sst_test.go +++ b/pkg/storage/sst_test.go @@ -74,7 +74,7 @@ func TestCheckSSTConflictsMaxIntents(t *testing.T) { require.NoError(t, batch.PutMVCC(MVCCKey{Key: roachpb.Key(key), Timestamp: txn1TS}, []byte("value"))) } for _, key := range intents { - require.NoError(t, MVCCPut(ctx, batch, nil, roachpb.Key(key), txn1TS, roachpb.MakeValueFromString("intent"), txn1)) + require.NoError(t, MVCCPut(ctx, batch, nil, roachpb.Key(key), txn1TS, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("intent"), txn1)) } require.NoError(t, batch.Commit(true)) batch.Close() From 24c56dffe866f4252556855731041bf4e070f573 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 3 Mar 2022 13:42:01 -0500 Subject: [PATCH 5/9] kv/storage: introduce local timestamps for MVCC versions in MVCCValue Fixes #36431. Fixes #49360. Replaces #72121. Replaces #77342. This commit fixes the potential for a stale read as detailed in #36431 using the "remember when intents were written" approach described in https://github.com/cockroachdb/cockroach/issues/36431#issuecomment-586134702 and later expanded on in https://github.com/cockroachdb/cockroach/pull/72121#issuecomment-954433047. This bug requires a combination of skewed clocks, multi-key transactions split across ranges whose leaseholders are stored on different nodes, a transaction read refresh, and the use of observed timestamps to avoid an uncertainty restart. With the combination of these four factors, it was possible to construct an ordering of events that violated real-time ordering and allowed a transaction to observe a stale read. Upon the discovery of the bug, we [introduced](https://github.com/cockroachdb/jepsen/pull/19) the `multi-register` test to the Jepsen test suite, and have since observed the test fail when combined with the `strobe-skews` nemesis due to this bug in #49360 (and a few issues linked to that one). This commit stabilizes that test. \### Explanation The combination of all of the factors listed above can lead to the stale read because it breaks one of the invariants that the observed timestamp infrastructure[^1] relied upon for correctness. Specifically, observed timestamps relied on the guarantee that a leaseholder's clock must always be equal to or greater than the version timestamp of all writes that it has served. However, this guarantee did not always hold. It does hold for non-transactional writes. It also holds for transactions that perform all of their intent writes at the same timestamp and then commit at this timestamp. However, it does not hold for transactions which move their commit timestamp forward over their lifetime before committing, writing intents at different timestamps along the way and "pulling them up" to the commit timestamp after committing. In violating the invariant, this third case reveals an ambiguity in what it means for a leaseholder to "serve a write at a timestamp". The meaning of this phrase is straightforward for non-transactional writes. However, for an intent write whose original timestamp is provisional and whose eventual commit timestamp is stored indirectly in its transaction record at its time of commit, the meaning is less clear. This reconciliation to move the intent write's timestamp up to its transaction's commit timestamp is asynchronous from the transaction commit (and after it has been externally acknowledged). So even if a leaseholder has only served writes with provisional timestamps up to timestamp 100 (placing a lower bound on its clock of 100), it can be in possession of intents that, when resolved, will carry a timestamp of 200. To uphold the real-time ordering property, this value must be observed by any transaction that begins after the value's transaction committed and was acknowledged. So for observed timestamps to be correct as currently written, we would need a guarantee that this value's leaseholder would never return an observed timestamp < 200 at any point after the transaction commits. But with the transaction commit possibly occurring on another node and with communication to resolve the intent occurring asynchronously, this seems like an impossible guarantee to make. This would appear to undermine observed timestamps to the point where they cannot be used. However, we can claw back correctness without sacrificing performance by recognizing that only a small fraction[^2] of transactions commit at a different timestamps than the one they used while writing intents. We can also recognize that if we were to compare observed timestamps against the timestamp that a committed value was originally written (its provisional value if it was once an intent) instead of the timestamp that it had been moved to on commit, then the invariant would hold. This commit exploits this second observation by adding a second timestamp to each MVCC key-value version called the "local timestamp". The existing version timestamp dictates the key-value's visibility to readers and is tied to the writer's commit timestamp. The local clock timestamp records the value of the local HLC clock on the leaseholder when the key was originally written. It is used to make claims about the relative real time ordering of the key's writer and readers when comparing a reader's uncertainty interval (and observed timestamps) to the key. Ignoring edge cases, readers with an observed timestamp from the key's leaseholder that is greater than the local clock timestamp stored in the key cannot make claims about real time ordering and must consider it possible that the key's write occurred before the read began. However, readers with an observed timestamp from the key's leaseholder that is less than the clock timestamp can claim that the reader captured that observed timestamp before the key was written and therefore can consider the key's write to have been concurrent with the read. In doing so, the reader can avoid an uncertainty restart. For more, see the updates made in this commit to pkg/kv/kvserver/observedts/doc.go. To avoid the bulk of the performance hit from adding this new timestamp to each key-value pair, the commit optimizes the clock timestamp away in the common case where it leads the version timestamp. Only in the rare cases where the local timestamp trails the version timestamp (e.g. future-time writes, async intent resolution with a new commit timestamp) does the local timestamp need to be explicitly represented in the key encoding. This is possible because it is safe for the local clock timestamp to be rounded down, as this will simply lead to additional uncertainty restarts. However, it is not safe for the local clock timestamp to be rounded up, as this could lead to stale reads. \### MVCCValue To store the local timestamp, the commit introduces a new MVCCValue type to parallel the MVCCKey type. MVCCValue wraps a roachpb.Value and extends it with MVCC-level metadata which is stored in an enginepb.MVCCValueHeader struct. To this point, the MVCC layer has treated versioned values as opaque blobs of bytes and has not enforced any structure on them. Now that MVCC will use the value to store metadata, it needs to enforce more structure on the values provided to it. This is the cause of some testing churn, but is otherwise not a problem, as all production code paths were already passing values in the roachpb.Value encoding. To further avoid any performance hit, MVCCValue has a "simple" and an "extended" encoding scheme, depending on whether the value's header is empty or not. If the value's header is empty, it is omitted in the encoding and the mvcc value's encoding is identical to that of roachpb.Value. This provided backwards compatibility and ensures that the MVCCValue optimizes away in the common case. If the value's header is not empty, it is prepended to the roachpb.Value encoding. The encoding scheme's variants are: ``` Simple (identical to the roachpb.Value encoding): <4-byte-checksum><1-byte-tag> Extended (header prepended to roachpb.Value encoding): <4-byte-header-len><1-byte-sentinel><4-byte-checksum><1-byte-tag> ``` The two encoding scheme variants are distinguished using the 5th byte, which is either the roachpb.Value tag (which has many possible values) or a sentinel tag not used by the roachpb.Value encoding which indicates the extended encoding scheme. Care was taken to ensure that encoding and decoding routines for the "simple" encoding are fast by avoiding heap allocations, memory copies, or function calls by exploiting mid-stack inlining. \### Future improvements As noted in https://github.com/cockroachdb/cockroach/pull/72121#issuecomment-954433047, this commit paves a path towards the complete removal of synthetic timestamps, which were originally introduced in support of non-blocking transactions and GLOBAL tables. The synthetic bit's first role of providing dynamic typing for `ClockTimestamps` is no longer necessary now that we never need to "push" transaction-domain timestamps into HLC clocks. Instead, the invariant that underpins observed timestamps is enforced by "pulling" local timestamps from the leaseholder's HLC clock. The synthetic bit's second role of disabling observed timestamps is replaced by the generalization provided by "local timestamps". Local timestamps precisely track when an MVCC version was written in the leaseholder's clock timestamp domain. This establishes a total ordering across clock observations (local timestamp assignment for writers and observed timestamps for readers) and establish a partial ordering between writer and reader transactions. As a result, the use of observed timestamps during uncertainty checking becomes a comparison between two `ClockTimestamps`, the version's local timestamp and the reader's observed timestamp. \### Correctness testing I was not able to stress `jepsen/multi-register/strobe-skews` hard enough to cause it to fail, even on master. We've only seen the test fail a handful of times over the past few years, so this isn't much of a surprise. Still, this prevents us from saying anything concrete about an reduced failure rate. However, the commit does add a new test called `TestTxnReadWithinUncertaintyIntervalAfterIntentResolution` which controls manual clocks directly and was able to deterministically reproduce the stale read before this fix in a few different ways. After this fix, the test passes. \### Performance analysis This correctness fix will lead to an increased rate of transaction retries under some workloads. TODO(nvanbenschoten): - microbenchmarks - single-process benchmarks - compare YCSB performance ---- Release note (bug fix): fixed a rare race condition that could allow for a transaction to serve a stale read and violate real-time ordering under moderate clock skew. [^1]: see [pkg/kv/kvserver/observedts/doc.go](https://github.com/cockroachdb/cockroach/blob/master/pkg/kv/kvserver/observedts/doc.go) for an explanation of the role of observed timestamps in the transaction model. This commit updates that documentation to include this fix. [^2]: see analysis in https://github.com/cockroachdb/cockroach/issues/36431#issuecomment-714221846. --- pkg/ccl/backupccl/backup_metadata.go | 8 +- pkg/ccl/backupccl/backup_processor.go | 2 +- pkg/ccl/backupccl/restore_data_processor.go | 5 +- .../backupccl/restore_data_processor_test.go | 25 +- pkg/cli/debug_check_store_test.go | 2 +- pkg/cmd/roachtest/tests/acceptance.go | 1 + pkg/kv/kvnemesis/engine.go | 23 +- pkg/kv/kvnemesis/validator.go | 20 +- pkg/kv/kvserver/batcheval/cmd_add_sstable.go | 10 +- .../batcheval/cmd_add_sstable_test.go | 6 +- pkg/kv/kvserver/client_replica_test.go | 247 ++ pkg/kv/kvserver/client_test.go | 9 + pkg/kv/kvserver/gc/data_distribution_test.go | 22 +- pkg/kv/kvserver/gc/gc_iterator_test.go | 4 +- pkg/kv/kvserver/rangefeed/catchup_scan.go | 11 +- .../kvserver/rangefeed/catchup_scan_test.go | 16 +- pkg/kv/kvserver/rangefeed/registry_test.go | 34 +- pkg/kv/kvserver/rangefeed/task_test.go | 14 +- pkg/kv/kvserver/replica_rangefeed_test.go | 14 +- pkg/kv/kvserver/spanset/batch.go | 9 +- pkg/kv/kvserver/uncertainty/doc.go | 170 +- pkg/kv/kvserver/uncertainty/interval.go | 23 +- pkg/kv/kvserver/uncertainty/interval_test.go | 110 +- pkg/roachpb/data.go | 1 + pkg/roachpb/data.proto | 6 + pkg/roachpb/errors.go | 3 + pkg/server/server_test.go | 5 +- pkg/storage/BUILD.bazel | 2 + pkg/storage/batch_test.go | 2 +- pkg/storage/bench_test.go | 18 +- pkg/storage/engine.go | 11 +- pkg/storage/engine_test.go | 3 +- pkg/storage/enginepb/mvcc.proto | 2 +- pkg/storage/enginepb/mvcc3.proto | 32 + pkg/storage/intent_interleaving_iter_test.go | 21 +- pkg/storage/multi_iterator_test.go | 2 +- pkg/storage/mvcc.go | 343 +- pkg/storage/mvcc_history_test.go | 38 +- pkg/storage/mvcc_incremental_iterator_test.go | 102 +- pkg/storage/mvcc_key.go | 11 +- pkg/storage/mvcc_stats_test.go | 59 +- pkg/storage/mvcc_test.go | 25 +- pkg/storage/mvcc_value.go | 245 ++ pkg/storage/mvcc_value_test.go | 236 ++ pkg/storage/pebble.go | 48 +- pkg/storage/pebble_batch.go | 14 +- pkg/storage/pebble_mvcc_scanner.go | 121 +- pkg/storage/pebble_mvcc_scanner_test.go | 27 +- pkg/storage/pebble_test.go | 45 +- pkg/storage/sst.go | 2 +- pkg/storage/sst_test.go | 12 +- pkg/storage/sst_writer.go | 17 +- .../conditional_put_with_txn_enable_separated | 4 +- .../testdata/mvcc_histories/ignored_seq_nums | 60 +- .../testdata/mvcc_histories/local_timestamp | 324 ++ ...inty_interval_with_local_uncertainty_limit | 686 ++-- ...uncertainty_limit_and_synthetic_timestamps | 3061 +++++++++++++++++ pkg/testutils/lint/lint_test.go | 2 + pkg/testutils/sstutil/kv.go | 6 +- pkg/testutils/sstutil/sstutil.go | 9 +- pkg/util/hlc/doc.go | 29 +- 61 files changed, 5499 insertions(+), 920 deletions(-) create mode 100644 pkg/storage/mvcc_value.go create mode 100644 pkg/storage/mvcc_value_test.go create mode 100644 pkg/storage/testdata/mvcc_histories/local_timestamp create mode 100644 pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps diff --git a/pkg/ccl/backupccl/backup_metadata.go b/pkg/ccl/backupccl/backup_metadata.go index fee3e840d639..9d3712465512 100644 --- a/pkg/ccl/backupccl/backup_metadata.go +++ b/pkg/ccl/backupccl/backup_metadata.go @@ -187,7 +187,7 @@ func writeDescsToMetadata(ctx context.Context, sst storage.SSTWriter, m *BackupM b = bytes } } - if err := sst.PutMVCC(storage.MVCCKey{Key: k, Timestamp: i.Time}, b); err != nil { + if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: i.Time}, b); err != nil { return err } @@ -214,7 +214,7 @@ func writeDescsToMetadata(ctx context.Context, sst storage.SSTWriter, m *BackupM return err } } else { - if err := sst.PutMVCC(storage.MVCCKey{Key: k, Timestamp: m.StartTime}, b); err != nil { + if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: m.StartTime}, b); err != nil { return err } } @@ -340,7 +340,7 @@ func writeNamesToMetadata(ctx context.Context, sst storage.SSTWriter, m *BackupM } k := encodeNameSSTKey(rev.parent, rev.parentSchema, rev.name) v := encoding.EncodeUvarintAscending(nil, uint64(rev.id)) - if err := sst.PutMVCC(storage.MVCCKey{Key: k, Timestamp: rev.ts}, v); err != nil { + if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: rev.ts}, v); err != nil { return err } } @@ -384,7 +384,7 @@ func writeSpansToMetadata(ctx context.Context, sst storage.SSTWriter, m *BackupM } } else { k := storage.MVCCKey{Key: encodeSpanSSTKey(sp), Timestamp: ts} - if err := sst.PutMVCC(k, nil); err != nil { + if err := sst.PutRawMVCC(k, nil); err != nil { return err } } diff --git a/pkg/ccl/backupccl/backup_processor.go b/pkg/ccl/backupccl/backup_processor.go index bc328d75aa7b..2120b7d9af9b 100644 --- a/pkg/ccl/backupccl/backup_processor.go +++ b/pkg/ccl/backupccl/backup_processor.go @@ -846,7 +846,7 @@ func (s *sstSink) write(ctx context.Context, resp returnedSST) error { return err } } else { - if err := s.sst.PutMVCC(sst.UnsafeKey(), sst.UnsafeValue()); err != nil { + if err := s.sst.PutRawMVCC(sst.UnsafeKey(), sst.UnsafeValue()); err != nil { return err } } diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index d95ef1b9bb38..6eaac8b9892c 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -476,9 +476,10 @@ func (rd *restoreDataProcessor) processRestoreSpanEntry( continue } - keyScratch = append(keyScratch[:0], iter.UnsafeKey().Key...) + key := iter.UnsafeKey() + keyScratch = append(keyScratch[:0], key.Key...) + key.Key = keyScratch valueScratch = append(valueScratch[:0], iter.UnsafeValue()...) - key := storage.MVCCKey{Key: keyScratch, Timestamp: iter.UnsafeKey().Timestamp} value := roachpb.Value{RawBytes: valueScratch} iter.NextKey() diff --git a/pkg/ccl/backupccl/restore_data_processor_test.go b/pkg/ccl/backupccl/restore_data_processor_test.go index ea2caed2b9a0..1c929cb5bc60 100644 --- a/pkg/ccl/backupccl/restore_data_processor_test.go +++ b/pkg/ccl/backupccl/restore_data_processor_test.go @@ -81,20 +81,23 @@ func slurpSSTablesLatestKey( if !sst.UnsafeKey().Less(end) { break } - var ok bool - var newKv storage.MVCCKeyValue key := sst.UnsafeKey() - newKv.Value = append(newKv.Value, sst.UnsafeValue()...) - newKv.Key.Key = append(newKv.Key.Key, key.Key...) - newKv.Key.Timestamp = key.Timestamp - newKv.Key.Key, ok = kr.rewriteKey(newKv.Key.Key) + value, err := storage.DecodeMVCCValue(sst.UnsafeValue()) + if err != nil { + t.Fatal(err) + } + newKey := key + newKey.Key = append([]byte(nil), newKey.Key...) + var ok bool + newKey.Key, ok = kr.rewriteKey(newKey.Key) if !ok { - t.Fatalf("could not rewrite key: %s", newKv.Key.Key) + t.Fatalf("could not rewrite key: %s", newKey.Key) } - v := roachpb.Value{RawBytes: newKv.Value} - v.ClearChecksum() - v.InitChecksum(newKv.Key.Key) - if err := batch.PutMVCC(newKv.Key, v.RawBytes); err != nil { + newValue := value + newValue.Value.RawBytes = append([]byte(nil), newValue.Value.RawBytes...) + newValue.Value.ClearChecksum() + newValue.Value.InitChecksum(newKey.Key) + if err := batch.PutMVCC(newKey, newValue); err != nil { t.Fatal(err) } sst.Next() diff --git a/pkg/cli/debug_check_store_test.go b/pkg/cli/debug_check_store_test.go index 0f1df93c5498..4153ef5049e8 100644 --- a/pkg/cli/debug_check_store_test.go +++ b/pkg/cli/debug_check_store_test.go @@ -75,7 +75,7 @@ func TestDebugCheckStore(t *testing.T) { // Should not error out randomly. for _, dir := range storePaths { out, err := check(dir) - require.NoError(t, err, dir) + require.NoError(t, err, "dir=%s\nout=%s\n", dir, out) require.Contains(t, out, "total stats", dir) } diff --git a/pkg/cmd/roachtest/tests/acceptance.go b/pkg/cmd/roachtest/tests/acceptance.go index d489c182be63..c0c45fb1efb1 100644 --- a/pkg/cmd/roachtest/tests/acceptance.go +++ b/pkg/cmd/roachtest/tests/acceptance.go @@ -52,6 +52,7 @@ func registerAcceptance(r registry.Registry) { }, { name: "version-upgrade", + skip: "WIP: unskip when version checks are added to local_timestamp writes", fn: func(ctx context.Context, t test.Test, c cluster.Cluster) { runVersionUpgrade(ctx, t, c) }, diff --git a/pkg/kv/kvnemesis/engine.go b/pkg/kv/kvnemesis/engine.go index f6cc5391c789..257acba4b4e3 100644 --- a/pkg/kv/kvnemesis/engine.go +++ b/pkg/kv/kvnemesis/engine.go @@ -67,12 +67,18 @@ func (e *Engine) Get(key roachpb.Key, ts hlc.Timestamp) roachpb.Value { if !mvccKey.Key.Equal(key) { return roachpb.Value{} } - if len(iter.Value()) == 0 { - return roachpb.Value{} - } var valCopy []byte e.b, valCopy = e.b.Copy(iter.Value(), 0 /* extraCap */) - return roachpb.Value{RawBytes: valCopy, Timestamp: mvccKey.Timestamp} + mvccVal, err := storage.DecodeMVCCValue(valCopy) + if err != nil { + panic(err) + } + if mvccVal.IsTombstone() { + return roachpb.Value{} + } + val := mvccVal.Value + val.Timestamp = mvccKey.Timestamp + return val } // Put inserts a key/value/timestamp tuple. If an exact key/timestamp pair is @@ -124,8 +130,13 @@ func (e *Engine) DebugPrint(indent string) string { if err != nil { fmt.Fprintf(&buf, "(err:%s)", err) } else { - fmt.Fprintf(&buf, "%s%s %s -> %s", - indent, key.Key, key.Timestamp, roachpb.Value{RawBytes: value}.PrettyPrint()) + v, err := storage.DecodeMVCCValue(value) + if err != nil { + fmt.Fprintf(&buf, "(err:%s)", err) + } else { + fmt.Fprintf(&buf, "%s%s %s -> %s", + indent, key.Key, key.Timestamp, v.Value.PrettyPrint()) + } } }) return buf.String() diff --git a/pkg/kv/kvnemesis/validator.go b/pkg/kv/kvnemesis/validator.go index 6ce9fab5e01b..c8d1c95b1523 100644 --- a/pkg/kv/kvnemesis/validator.go +++ b/pkg/kv/kvnemesis/validator.go @@ -273,8 +273,12 @@ func makeValidator(kvs *Engine) (*validator, error) { err = errors.CombineErrors(err, iterErr) return } - v := roachpb.Value{RawBytes: value} - if v.GetTag() != roachpb.ValueType_UNKNOWN { + v, decodeErr := storage.DecodeMVCCValue(value) + if err != nil { + err = errors.CombineErrors(err, decodeErr) + return + } + if v.Value.GetTag() != roachpb.ValueType_UNKNOWN { valueStr := mustGetStringValue(value) if existing, ok := kvByValue[valueStr]; ok { // TODO(dan): This may be too strict. Some operations (db.Run on a @@ -287,7 +291,7 @@ func makeValidator(kvs *Engine) (*validator, error) { // globally over a run, so there's a 1:1 relationship between a value that // was written and the operation that wrote it. kvByValue[valueStr] = storage.MVCCKeyValue{Key: key, Value: value} - } else if len(value) == 0 { + } else if !v.Value.IsPresent() { rawKey := string(key.Key) if _, ok := tombstonesForKey[rawKey]; !ok { tombstonesForKey[rawKey] = make(map[hlc.Timestamp]bool) @@ -914,14 +918,18 @@ func resultIsErrorStr(r Result, msgRE string) bool { } func mustGetStringValue(value []byte) string { - if len(value) == 0 { + v, err := storage.DecodeMVCCValue(value) + if err != nil { + panic(errors.Wrapf(err, "decoding %x", value)) + } + if v.IsTombstone() { return `` } - v, err := roachpb.Value{RawBytes: value}.GetBytes() + b, err := v.Value.GetBytes() if err != nil { panic(errors.Wrapf(err, "decoding %x", value)) } - return string(v) + return string(b) } func validReadTimes( diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index 75cd43a52c0e..4c0eba446b09 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -335,7 +335,7 @@ func EvalAddSSTable( return result.Result{}, err } } else { - if err := readWriter.PutMVCC(k, sstIter.UnsafeValue()); err != nil { + if err := readWriter.PutRawMVCC(k, sstIter.UnsafeValue()); err != nil { return result.Result{}, err } } @@ -401,11 +401,15 @@ func assertSSTContents(sst []byte, sstTimestamp hlc.Timestamp, stats *enginepb.M break } - key, value := iter.UnsafeKey(), iter.UnsafeValue() + key, valueRaw := iter.UnsafeKey(), iter.UnsafeValue() + value, err := storage.DecodeMVCCValue(valueRaw) + if err != nil { + return err + } if key.Timestamp.IsEmpty() { return errors.AssertionFailedf("SST contains inline value or intent for key %s", key) } - if len(value) == 0 { + if value.IsTombstone() { return errors.AssertionFailedf("SST contains tombstone for key %s", key) } if sstTimestamp.IsSet() && key.Timestamp != sstTimestamp { diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 7635e89a539e..b29e721d04bf 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -1117,7 +1117,7 @@ func TestAddSSTableMVCCStats(t *testing.T) { {"e", 1, "e"}, {"z", 2, "zzzzzz"}, } { - require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) } sst, start, end := sstutil.MakeSST(t, st, []sstutil.KV{ @@ -1228,7 +1228,7 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { {"y", 5, "yyy"}, {"z", 2, "zz"}, } { - require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) } // This test ensures accuracy of MVCCStats in the situation that successive @@ -1270,7 +1270,7 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { // ingesting the perfectly shadowing KVs (same ts and same value) in the // second SST. for _, kv := range kvs { - require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) } // Evaluate the second SST. Both the KVs are perfectly shadowing and should diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 85a73f746266..31216da2153d 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -492,6 +492,253 @@ func TestTxnReadWithinUncertaintyInterval(t *testing.T) { }) } +// TestTxnReadWithinUncertaintyIntervalAfterIntentResolution tests cases where a +// reader transaction observes a committed value that was committed before the +// reader began, but that was resolved after the reader began. The test ensures +// that even if the reader has collected an observed timestamp from the node +// that holds the intent, and even if this observed timestamp is less than the +// timestamp that the intent is eventually committed at, the reader still +// considers the value to be in its uncertainty interval. Not doing so could +// allow for stale read, which would be a violation of linearizability. +// +// This is a regression test for #36431. Before this issue was addressed, +// it was possible for the following series of events to lead to a stale +// read: +// - txn W is coordinated by node B. It lays down an intent on node A (key k) at +// ts 95. +// - txn W gets pushed to ts 105 (taken from B's clock). It refreshes +// successfully and commits at 105. Node A's clock is at, say, 100; this is +// within clock offset bounds. +// - after all this, txn R starts on node A. It gets assigned ts 100. The txn +// has no uncertainty for node A. +// - txn W's async intent resolution comes around and resolves the intent on +// node A, moving the value fwd from ts 95 to 105. +// - txn R reads key k and doesn't see anything. There's a value at 105, but the +// txn have no uncertainty due to an observed timestamp. This is a stale read. +// +// The test's rangedResolution parameter dictates whether the intent is +// asynchronously resolved using point or ranged intent resolution. +// +// The test's movedWhilePending parameter dictates whether the intent is moved +// to a higher timestamp first by a PENDING intent resolution and then COMMITTED +// at that same timestamp, or whether it is moved to a higher timestamp at the +// same time as it is COMMITTED. +// +// The test's alreadyResolved parameter dictates whether the intent is +// already resolved by the time the reader observes it, or whether the +// reader must resolve the intent itself. +// +func TestTxnReadWithinUncertaintyIntervalAfterIntentResolution(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunTrueAndFalse(t, "rangedResolution", func(t *testing.T, rangedResolution bool) { + testutils.RunTrueAndFalse(t, "movedWhilePending", func(t *testing.T, movedWhilePending bool) { + testutils.RunTrueAndFalse(t, "alreadyResolved", func(t *testing.T, alreadyResolved bool) { + testTxnReadWithinUncertaintyIntervalAfterIntentResolution( + t, rangedResolution, movedWhilePending, alreadyResolved, + ) + }) + }) + }) +} + +func testTxnReadWithinUncertaintyIntervalAfterIntentResolution( + t *testing.T, rangedResolution, movedWhilePending, alreadyResolved bool, +) { + const numNodes = 2 + var manuals []*hlc.HybridManualClock + var clocks []*hlc.Clock + for i := 0; i < numNodes; i++ { + manuals = append(manuals, hlc.NewHybridManualClock()) + } + serverArgs := make(map[int]base.TestServerArgs) + for i := 0; i < numNodes; i++ { + serverArgs[i] = base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manuals[i].UnixNano, + }, + Store: &kvserver.StoreTestingKnobs{ + IntentResolverKnobs: kvserverbase.IntentResolverTestingKnobs{ + // Disable async intent resolution, so that the test can carefully + // control when intent resolution occurs. + DisableAsyncIntentResolution: true, + }, + }, + }, + } + } + ctx := context.Background() + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: serverArgs, + }) + defer tc.Stopper().Stop(ctx) + + // Split off two scratch ranges. + keyA, keyB := roachpb.Key("a"), roachpb.Key("b") + tc.SplitRangeOrFatal(t, keyA) + _, keyBDesc := tc.SplitRangeOrFatal(t, keyB) + // Place key A's sole replica on node 1 and key B's sole replica on node 2. + tc.AddVotersOrFatal(t, keyB, tc.Target(1)) + tc.TransferRangeLeaseOrFatal(t, keyBDesc, tc.Target(1)) + tc.RemoveVotersOrFatal(t, keyB, tc.Target(0)) + + // Pause the servers' clocks going forward. + var maxNanos int64 + for i, m := range manuals { + m.Pause() + if cur := m.UnixNano(); cur > maxNanos { + maxNanos = cur + } + clocks = append(clocks, tc.Servers[i].Clock()) + } + // After doing so, perfectly synchronize them. + for _, m := range manuals { + m.Increment(maxNanos - m.UnixNano()) + } + + // Create a new writer transaction. + maxOffset := clocks[0].MaxOffset().Nanoseconds() + require.NotZero(t, maxOffset) + writerTxn := roachpb.MakeTransaction("test_writer", keyA, 1, clocks[0].Now(), maxOffset, int32(tc.Servers[0].NodeID())) + + // Write to key A and key B in the writer transaction. + for _, key := range []roachpb.Key{keyA, keyB} { + put := putArgs(key, []byte("val")) + resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), roachpb.Header{Txn: &writerTxn}, put) + require.Nil(t, pErr) + writerTxn.Update(resp.Header().Txn) + } + + // Move the clock on just the first server and bump the transaction commit + // timestamp to this value. The clock on the second server will trail behind. + manuals[0].Increment(100) + require.True(t, writerTxn.WriteTimestamp.Forward(clocks[0].Now())) + + // Refresh the writer transaction's timestamp. + writerTxn.ReadTimestamp.Forward(writerTxn.WriteTimestamp) + + // Commit the writer transaction. Key A will be synchronously resolved because + // it is on the same range as the transaction record. However, key B will be + // handed to the IntentResolver for asynchronous resolution. Because we + // disabled async resolution, it will not be resolved yet. + et, etH := endTxnArgs(&writerTxn, true /* commit */) + et.LockSpans = []roachpb.Span{ + {Key: keyA}, {Key: keyB}, + } + if rangedResolution { + for i := range et.LockSpans { + et.LockSpans[i].EndKey = et.LockSpans[i].Key.Next() + } + } + etResp, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), etH, et) + require.Nil(t, pErr) + writerTxn.Update(etResp.Header().Txn) + + // Create a new reader transaction. The reader uses the second server as a + // gateway, so its initial read timestamp actually trails the commit timestamp + // of the writer transaction due to clock skew between the two servers. This + // is the classic case where the reader's uncertainty interval is needed to + // avoid stale reads. Remember that the reader transaction began after the + // writer transaction committed and received an ack, so it must observe the + // writer's writes if it is to respect real-time ordering. + // + // NB: we use writerTxn.MinTimestamp instead of clocks[1].Now() so that a + // stray clock update doesn't influence the reader's read timestamp. + readerTxn := roachpb.MakeTransaction("test_reader", keyA, 1, writerTxn.MinTimestamp, maxOffset, int32(tc.Servers[1].NodeID())) + require.True(t, readerTxn.ReadTimestamp.Less(writerTxn.WriteTimestamp)) + require.False(t, readerTxn.GlobalUncertaintyLimit.Less(writerTxn.WriteTimestamp)) + + // Collect an observed timestamp from each of the nodes. We read the key + // following (Key.Next) each of the written keys to avoid conflicting with + // read values. We read keyB first to avoid advancing the clock on node 2 + // before we collect an observed timestamp from it. + // + // NOTE: this wasn't even a necessary step to hit #36431, because new + // transactions are always an observed timestamp from their own gateway node. + for i, key := range []roachpb.Key{keyB, keyA} { + get := getArgs(key.Next()) + resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), roachpb.Header{Txn: &readerTxn}, get) + require.Nil(t, pErr) + require.Nil(t, resp.(*roachpb.GetResponse).Value) + readerTxn.Update(resp.Header().Txn) + require.Len(t, readerTxn.ObservedTimestamps, i+1) + } + + // Resolve the intent on key B zero, one, or two times. + { + resolveIntentArgs := func(status roachpb.TransactionStatus) roachpb.Request { + if rangedResolution { + return &roachpb.ResolveIntentRangeRequest{ + RequestHeader: roachpb.RequestHeader{Key: keyB, EndKey: keyB.Next()}, + IntentTxn: writerTxn.TxnMeta, + Status: status, + } + } else { + return &roachpb.ResolveIntentRequest{ + RequestHeader: roachpb.RequestHeader{Key: keyB}, + IntentTxn: writerTxn.TxnMeta, + Status: status, + } + } + } + + if movedWhilePending { + // First change the intent's timestamp without committing it. This + // exercises the case where the intent's timestamp is moved forward by a + // PENDING intent resolution request and kept the same when the intent is + // eventually COMMITTED. This PENDING intent resolution may still be + // evaluated after the transaction commit has been acknowledged in + // real-time, so it still needs to lead to the committed value retaining + // its original local timestamp. + // + // For instance, consider the following timeline: + // + // 1. txn W writes intent on key A @ time 10 + // 2. txn W writes intent on key B @ time 10 + // 3. high priority reader @ 15 reads key B + // 4. high priority reader pushes txn W to time 15 + // 5. txn W commits @ 15 and resolves key A synchronously + // 6. txn R begins and collects observed timestamp from key B's node @ + // time 11 + // 7. high priority reader moves intent on key B to time 15 + // 8. async intent resolution commits intent on key B, still @ time 15 + // 9. txn R reads key B with read ts 11, observed ts 11, and uncertainty + // interval [11, 21]. If step 7 updated the intent's local timestamp + // to the current time when changing its version timestamp, txn R + // could use its observed timestamp to avoid an uncertainty error, + // leading to a stale read. + // + resolve := resolveIntentArgs(roachpb.PENDING) + _, pErr = kv.SendWrapped(ctx, tc.Servers[0].DistSender(), resolve) + require.Nil(t, pErr) + } + + if alreadyResolved { + // Resolve the committed value on key B to COMMITTED. + resolve := resolveIntentArgs(roachpb.COMMITTED) + _, pErr = kv.SendWrapped(ctx, tc.Servers[0].DistSender(), resolve) + require.Nil(t, pErr) + } + } + + // Read key A and B in the reader transaction. Both should produce + // ReadWithinUncertaintyIntervalErrors. + for _, key := range []roachpb.Key{keyA, keyB} { + get := getArgs(key) + _, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), roachpb.Header{Txn: &readerTxn}, get) + require.NotNil(t, pErr) + var rwuiErr *roachpb.ReadWithinUncertaintyIntervalError + require.True(t, errors.As(pErr.GetDetail(), &rwuiErr)) + require.Equal(t, readerTxn.ReadTimestamp, rwuiErr.ReadTimestamp) + require.Equal(t, readerTxn.GlobalUncertaintyLimit, rwuiErr.GlobalUncertaintyLimit) + require.Equal(t, readerTxn.ObservedTimestamps, rwuiErr.ObservedTimestamps) + require.Equal(t, writerTxn.WriteTimestamp, rwuiErr.ExistingTimestamp) + } +} + // TestTxnReadWithinUncertaintyIntervalAfterLeaseTransfer tests a case where a // transaction observes a committed value in its uncertainty interval that was // written under a previous leaseholder. In the test, the transaction does diff --git a/pkg/kv/kvserver/client_test.go b/pkg/kv/kvserver/client_test.go index 54405929f758..a300b9dcdc4b 100644 --- a/pkg/kv/kvserver/client_test.go +++ b/pkg/kv/kvserver/client_test.go @@ -102,6 +102,15 @@ func heartbeatArgs( }, roachpb.Header{Txn: txn} } +func endTxnArgs(txn *roachpb.Transaction, commit bool) (*roachpb.EndTxnRequest, roachpb.Header) { + return &roachpb.EndTxnRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: txn.Key, // not allowed when going through TxnCoordSender, but we're not + }, + Commit: commit, + }, roachpb.Header{Txn: txn} +} + func pushTxnArgs( pusher, pushee *roachpb.Transaction, pushType roachpb.PushTxnType, ) *roachpb.PushTxnRequest { diff --git a/pkg/kv/kvserver/gc/data_distribution_test.go b/pkg/kv/kvserver/gc/data_distribution_test.go index f032b7eb120d..ac4945c42969 100644 --- a/pkg/kv/kvserver/gc/data_distribution_test.go +++ b/pkg/kv/kvserver/gc/data_distribution_test.go @@ -52,7 +52,7 @@ func (ds dataDistribution) setupTest( if kv.Key.Timestamp.IsEmpty() { require.NoError(t, eng.PutUnversioned(kv.Key.Key, kv.Value)) } else { - require.NoError(t, eng.PutMVCC(kv.Key, kv.Value)) + require.NoError(t, eng.PutRawMVCC(kv.Key, kv.Value)) } } else { // TODO(ajwerner): Decide if using MVCCPut is worth it. @@ -84,7 +84,7 @@ func (ds dataDistribution) setupTest( func newDataDistribution( tsDist func() hlc.Timestamp, keyDist func() roachpb.Key, - valueDist func() []byte, + valueDist func() roachpb.Value, versionsPerKey func() int, intentFrac float64, totalKeys int, @@ -146,7 +146,7 @@ func newDataDistribution( } return storage.MVCCKeyValue{ Key: storage.MVCCKey{Key: key, Timestamp: ts}, - Value: valueDist(), + Value: valueDist().RawBytes, }, txn, true } } @@ -218,20 +218,24 @@ func uniformTimestampDistribution(from, to int64, rng *rand.Rand) func() hlc.Tim } // returns a uniform length random value distribution. -func uniformValueDistribution(min, max int, deleteFrac float64, rng *rand.Rand) func() []byte { +func uniformValueDistribution( + min, max int, deleteFrac float64, rng *rand.Rand, +) func() roachpb.Value { if min > max { panic(fmt.Errorf("min (%d) > max (%d)", min, max)) } n := (max - min) + 1 - return func() []byte { + return func() roachpb.Value { if rng.Float64() < deleteFrac { - return nil + return roachpb.Value{} } - value := make([]byte, min+rng.Intn(n)) - if _, err := rng.Read(value); err != nil { + b := make([]byte, min+rng.Intn(n)) + if _, err := rng.Read(b); err != nil { panic(err) } - return value + var v roachpb.Value + v.SetBytes(b) + return v } } diff --git a/pkg/kv/kvserver/gc/gc_iterator_test.go b/pkg/kv/kvserver/gc/gc_iterator_test.go index 2f4533436a1e..8c148f30d82e 100644 --- a/pkg/kv/kvserver/gc/gc_iterator_test.go +++ b/pkg/kv/kvserver/gc/gc_iterator_test.go @@ -35,14 +35,14 @@ func TestGCIterator(t *testing.T) { txn *roachpb.Transaction } // makeDataItem is a shorthand to construct dataItems. - makeDataItem := func(k roachpb.Key, val []byte, ts int64, txn *roachpb.Transaction) dataItem { + makeDataItem := func(k roachpb.Key, val roachpb.Value, ts int64, txn *roachpb.Transaction) dataItem { return dataItem{ MVCCKeyValue: storage.MVCCKeyValue{ Key: storage.MVCCKey{ Key: k, Timestamp: hlc.Timestamp{WallTime: ts * time.Nanosecond.Nanoseconds()}, }, - Value: val, + Value: val.RawBytes, }, txn: txn, } diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan.go b/pkg/kv/kvserver/rangefeed/catchup_scan.go index 2f30a6d04087..357d4c63cfb7 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan.go @@ -157,10 +157,11 @@ func (i *CatchUpIterator) CatchUpScan( } unsafeKey := i.UnsafeKey() - unsafeVal := i.UnsafeValue() + unsafeValRaw := i.UnsafeValue() + var unsafeVal []byte if !unsafeKey.IsValue() { // Found a metadata key. - if err := protoutil.Unmarshal(unsafeVal, &meta); err != nil { + if err := protoutil.Unmarshal(unsafeValRaw, &meta); err != nil { return errors.Wrapf(err, "unmarshaling mvcc meta: %v", unsafeKey) } if !meta.IsInline() { @@ -195,6 +196,12 @@ func (i *CatchUpIterator) CatchUpScan( // iterators may result in the rangefeed not // seeing some intermediate values. unsafeVal = meta.RawBytes + } else { + mvccVal, err := storage.DecodeMVCCValue(unsafeValRaw) + if err != nil { + return errors.Wrapf(err, "decoding mvcc value: %v", unsafeKey) + } + unsafeVal = mvccVal.Value.RawBytes } // Ignore the version if it's not inline and its timestamp is at diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go index 184b509aa6ee..7991581958a1 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go @@ -34,10 +34,10 @@ func TestCatchupScan(t *testing.T) { testKey1 = roachpb.Key("/db1") testKey2 = roachpb.Key("/db2") - testValue1 = []byte("val1") - testValue2 = []byte("val2") - testValue3 = []byte("val3") - testValue4 = []byte("val4") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") + testValue3 = roachpb.MakeValueFromString("val3") + testValue4 = roachpb.MakeValueFromString("val4") ts1 = hlc.Timestamp{WallTime: 1, Logical: 0} ts2 = hlc.Timestamp{WallTime: 2, Logical: 0} @@ -46,7 +46,7 @@ func TestCatchupScan(t *testing.T) { ts5 = hlc.Timestamp{WallTime: 4, Logical: 0} ) - makeTxn := func(key roachpb.Key, val []byte, ts hlc.Timestamp, + makeTxn := func(key roachpb.Key, val roachpb.Value, ts hlc.Timestamp, ) (roachpb.Transaction, roachpb.Value) { txnID := uuid.MakeV4() txnMeta := enginepb.TxnMeta{ @@ -59,12 +59,12 @@ func TestCatchupScan(t *testing.T) { TxnMeta: txnMeta, ReadTimestamp: ts, }, roachpb.Value{ - RawBytes: val, + RawBytes: val.RawBytes, } } - makeKTV := func(key roachpb.Key, ts hlc.Timestamp, value []byte) storage.MVCCKeyValue { - return storage.MVCCKeyValue{Key: storage.MVCCKey{Key: key, Timestamp: ts}, Value: value} + makeKTV := func(key roachpb.Key, ts hlc.Timestamp, value roachpb.Value) storage.MVCCKeyValue { + return storage.MVCCKeyValue{Key: storage.MVCCKey{Key: key, Timestamp: ts}, Value: value.RawBytes} } // testKey1 has an intent and provisional value that will be skipped. Both // testKey1 and testKey2 have a value that is older than what we need with diff --git a/pkg/kv/kvserver/rangefeed/registry_test.go b/pkg/kv/kvserver/rangefeed/registry_test.go index 315ed7746710..a2cd8f243f11 100644 --- a/pkg/kv/kvserver/rangefeed/registry_test.go +++ b/pkg/kv/kvserver/rangefeed/registry_test.go @@ -286,50 +286,50 @@ func TestRegistrationCatchUpScan(t *testing.T) { expEvents := []*roachpb.RangeFeedEvent{ rangeFeedValueWithPrev( roachpb.Key("d"), - roachpb.Value{RawBytes: []byte("valD3"), Timestamp: hlc.Timestamp{WallTime: 16}}, - roachpb.Value{RawBytes: []byte("valD2")}, + makeValWithTs("valD3", 16), + makeVal("valD2"), ), rangeFeedValueWithPrev( roachpb.Key("d"), - roachpb.Value{RawBytes: []byte("valD4"), Timestamp: hlc.Timestamp{WallTime: 19}}, - roachpb.Value{RawBytes: []byte("valD3")}, + makeValWithTs("valD4", 19), + makeVal("valD3"), ), rangeFeedValueWithPrev( roachpb.Key("d"), - roachpb.Value{RawBytes: []byte("valD5"), Timestamp: hlc.Timestamp{WallTime: 20}}, - roachpb.Value{RawBytes: []byte("valD4")}, + makeValWithTs("valD5", 20), + makeVal("valD4"), ), rangeFeedValueWithPrev( roachpb.Key("e"), - roachpb.Value{RawBytes: []byte("valE2"), Timestamp: hlc.Timestamp{WallTime: 5}}, - roachpb.Value{RawBytes: []byte("valE1")}, + makeValWithTs("valE2", 5), + makeVal("valE1"), ), rangeFeedValueWithPrev( roachpb.Key("e"), - roachpb.Value{RawBytes: []byte("valE3"), Timestamp: hlc.Timestamp{WallTime: 6}}, - roachpb.Value{RawBytes: []byte("valE2")}, + makeValWithTs("valE3", 6), + makeVal("valE2"), ), rangeFeedValue( roachpb.Key("f"), - roachpb.Value{RawBytes: []byte("valF1"), Timestamp: hlc.Timestamp{WallTime: 5}}, + makeValWithTs("valF1", 5), ), rangeFeedValueWithPrev( roachpb.Key("f"), - roachpb.Value{RawBytes: []byte("valF2"), Timestamp: hlc.Timestamp{WallTime: 6}}, - roachpb.Value{RawBytes: []byte("valF1")}, + makeValWithTs("valF2", 6), + makeVal("valF1"), ), rangeFeedValueWithPrev( roachpb.Key("f"), - roachpb.Value{RawBytes: []byte("valF3"), Timestamp: hlc.Timestamp{WallTime: 7}}, - roachpb.Value{RawBytes: []byte("valF2")}, + makeValWithTs("valF3", 7), + makeVal("valF2"), ), rangeFeedValue( roachpb.Key("g"), - roachpb.Value{RawBytes: []byte("valG1"), Timestamp: hlc.Timestamp{WallTime: 0}}, + makeVal("valG1"), ), rangeFeedValue( roachpb.Key("h"), - roachpb.Value{RawBytes: []byte("valH1"), Timestamp: hlc.Timestamp{WallTime: 15}}, + makeValWithTs("valH1", 15), ), } require.Equal(t, expEvents, r.Events()) diff --git a/pkg/kv/kvserver/rangefeed/task_test.go b/pkg/kv/kvserver/rangefeed/task_test.go index 9c87c4f93e40..45971de7c0c3 100644 --- a/pkg/kv/kvserver/rangefeed/task_test.go +++ b/pkg/kv/kvserver/rangefeed/task_test.go @@ -27,13 +27,23 @@ import ( "github.com/stretchr/testify/require" ) +func makeVal(val string) roachpb.Value { + return roachpb.MakeValueFromString(val) +} + +func makeValWithTs(val string, ts int64) roachpb.Value { + v := makeVal(val) + v.Timestamp = hlc.Timestamp{WallTime: ts} + return v +} + func makeKV(key, val string, ts int64) storage.MVCCKeyValue { return storage.MVCCKeyValue{ Key: storage.MVCCKey{ Key: roachpb.Key(key), Timestamp: hlc.Timestamp{WallTime: ts}, }, - Value: []byte(val), + Value: makeVal(val).RawBytes, } } @@ -56,7 +66,7 @@ func makeMetaKV(key string, meta enginepb.MVCCMetadata) storage.MVCCKeyValue { func makeInline(key, val string) storage.MVCCKeyValue { return makeMetaKV(key, enginepb.MVCCMetadata{ - RawBytes: []byte(val), + RawBytes: makeVal(val).RawBytes, }) } diff --git a/pkg/kv/kvserver/replica_rangefeed_test.go b/pkg/kv/kvserver/replica_rangefeed_test.go index 89b58da9f1c8..9a6960c99b94 100644 --- a/pkg/kv/kvserver/replica_rangefeed_test.go +++ b/pkg/kv/kvserver/replica_rangefeed_test.go @@ -341,10 +341,10 @@ func TestReplicaRangefeed(t *testing.T) { defer sstWriter.Close() require.NoError(t, sstWriter.PutMVCC( storage.MVCCKey{Key: roachpb.Key("b"), Timestamp: ts6}, - expVal6b.RawBytes)) + storage.MVCCValue{Value: expVal6b})) require.NoError(t, sstWriter.PutMVCC( storage.MVCCKey{Key: roachpb.Key("q"), Timestamp: ts6}, - expVal6q.RawBytes)) + storage.MVCCValue{Value: expVal6q})) require.NoError(t, sstWriter.Finish()) expSST := sstFile.Data() expSSTSpan := roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("r")} @@ -357,11 +357,11 @@ func TestReplicaRangefeed(t *testing.T) { // Ingest an SSTable as writes. ts7 := ts.Clock().Now().Add(0, 7) - expVal7b := roachpb.Value{Timestamp: ts7} + expVal7b := roachpb.Value{} expVal7b.SetInt(7) expVal7b.InitChecksum(roachpb.Key("b")) - expVal7q := roachpb.Value{Timestamp: ts7} + expVal7q := roachpb.Value{} expVal7q.SetInt(7) expVal7q.InitChecksum(roachpb.Key("q")) @@ -370,10 +370,10 @@ func TestReplicaRangefeed(t *testing.T) { defer sstWriter.Close() require.NoError(t, sstWriter.PutMVCC( storage.MVCCKey{Key: roachpb.Key("b"), Timestamp: ts7}, - expVal7b.RawBytes)) + storage.MVCCValue{Value: expVal7b})) require.NoError(t, sstWriter.PutMVCC( storage.MVCCKey{Key: roachpb.Key("q"), Timestamp: ts7}, - expVal7q.RawBytes)) + storage.MVCCValue{Value: expVal7q})) require.NoError(t, sstWriter.Finish()) _, _, _, pErr = store1.DB().AddSSTableAtBatchTimestamp(ctx, roachpb.Key("b"), roachpb.Key("r"), sstFile.Data(), @@ -391,6 +391,8 @@ func TestReplicaRangefeed(t *testing.T) { expVal5 := roachpb.Value{Timestamp: ts5} expVal5.SetInt(25) expVal5.InitChecksum(roachpb.Key("b")) + expVal7b.Timestamp = ts7 + expVal7q.Timestamp = ts7 expVal1NoTS, expVal4NoTS := expVal1, expVal4 expVal1NoTS.Timestamp, expVal4NoTS.Timestamp = hlc.Timestamp{}, hlc.Timestamp{} expEvents = append(expEvents, []*roachpb.RangeFeedEvent{ diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index ae012cb87a87..a8ac49d22083 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -612,13 +612,20 @@ func (s spanSetWriter) Merge(key storage.MVCCKey, value []byte) error { return s.w.Merge(key, value) } -func (s spanSetWriter) PutMVCC(key storage.MVCCKey, value []byte) error { +func (s spanSetWriter) PutMVCC(key storage.MVCCKey, value storage.MVCCValue) error { if err := s.checkAllowed(key.Key); err != nil { return err } return s.w.PutMVCC(key, value) } +func (s spanSetWriter) PutRawMVCC(key storage.MVCCKey, value []byte) error { + if err := s.checkAllowed(key.Key); err != nil { + return err + } + return s.w.PutRawMVCC(key, value) +} + func (s spanSetWriter) PutUnversioned(key roachpb.Key, value []byte) error { if err := s.checkAllowed(key); err != nil { return err diff --git a/pkg/kv/kvserver/uncertainty/doc.go b/pkg/kv/kvserver/uncertainty/doc.go index 81fc80dc6941..c4b431b871c5 100644 --- a/pkg/kv/kvserver/uncertainty/doc.go +++ b/pkg/kv/kvserver/uncertainty/doc.go @@ -20,10 +20,69 @@ // uncertainty related restarts. package uncertainty -import "github.com/cockroachdb/cockroach/pkg/roachpb" +import ( + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" +) + +// D0 ———————————————————————————————————————————————— +// +// MVCCKey +// +// Each value in CockroachDB is stored at an associated versioned key. This key +// is distinguished from roachpb.Key with the addition of a "version" timestamp. +// +// As a multi-version concurrency control (MVCC) system, the version timestamp +// dictates the key's visibility to readers. Readers with read timestamps equal +// to or greater than the version timestamp observe the key. Readers with read +// timestamps below the version timestamp ignore the key. +// +// NOTE: inlined struct definition to avoid circular dependency. +var D0 = /* storage.MVCCKey */ struct { + Key roachpb.Key + Timestamp hlc.Timestamp +}{} // D1 ———————————————————————————————————————————————— // +// MVCCValue +// +// At and below the MVCC layer, each roachpb.Value as wrapped with a header that +// contains MVCC-level metadata. One piece of metadata stored in this header is +// the value's "local" clock timestamp. +// +// The local clock timestamp records the value of the local HLC clock on the +// leaseholder when the key-value was originally written. It is used to make +// claims about the relative real time ordering of the key-value's writer and +// readers. +// +// When a versioned key-value is encountered by a transaction while reading, +// there are three possible outcomes: +// 1. the key's value is visible to the reader and is returned. +// 2. the key's value is not visible to the reader but is determined to have +// definitely been written concurrently with reader's transaction, so it is +// ignored and not returned. +// 3. the key's value is not visible to the reader but is determined to have +// been possibly written before the reader's transaction, so the reader must +// perform an uncertainty restart and later return the value to ensure real- +// time ordering guarantees. +// +// We call the differentiation between outcomes 2 and 3 "uncertainty". The rest +// of this package explores the mechanisms used to guarantee real-time ordering +// between transactions while attempting to maximize the number of readers that +// end up in outcome 2 and minimize the number that end up in outcome 3, thereby +// minimizing uncertainty restarts and maximizing concurrency. +// +// NOTE: inlined struct definition to avoid circular dependency. +var D1 = /* storage.MVCCValue */ struct { + Header struct { + LocalTimestamp hlc.ClockTimestamp + } + Value roachpb.Value +}{} + +// D2 ———————————————————————————————————————————————— +// // Transaction.GlobalUncertaintyLimit // // A transaction's global uncertainty limit is the inclusive upper bound of its @@ -32,9 +91,9 @@ import "github.com/cockroachdb/cockroach/pkg/roachpb" // respected, this maximum timestamp places an upper bound on the commit // timestamp of any other transaction that committed causally before the new // transaction. -var D1 = roachpb.Transaction{}.GlobalUncertaintyLimit +var D2 = roachpb.Transaction{}.GlobalUncertaintyLimit -// D2 ———————————————————————————————————————————————— +// D3 ———————————————————————————————————————————————— // // Interval // @@ -43,9 +102,9 @@ var D1 = roachpb.Transaction{}.GlobalUncertaintyLimit // global uncertainty limit is pulled directly from the transaction. The local // uncertainty limit is an optional tighter bound established through HLC clock // observations on individual nodes in the system. -var D2 = Interval{} +var D3 = Interval{} -// D3 ———————————————————————————————————————————————— +// D4 ———————————————————————————————————————————————— // // ReadWithinUncertaintyIntervalError // @@ -59,9 +118,9 @@ var D2 = Interval{} // guaranteed to observe any value written by any other transaction with a // happened-before relation to it, which is paramount to ensure single-key // linearizability and avoid stale reads. -var D3 = roachpb.ReadWithinUncertaintyIntervalError{} +var D4 = roachpb.ReadWithinUncertaintyIntervalError{} -// D4 ———————————————————————————————————————————————— +// D5 ———————————————————————————————————————————————— // // ObservedTimestamp // @@ -73,17 +132,17 @@ var D3 = roachpb.ReadWithinUncertaintyIntervalError{} // promise that any later clock reading from the same node will have a larger // value. Similarly, they are a promise that any prior update to the clock was // given a smaller value. -var D4 = roachpb.ObservedTimestamp{} +var D5 = roachpb.ObservedTimestamp{} -// D5 ———————————————————————————————————————————————— +// D6 ———————————————————————————————————————————————— // // Transaction.UpdateObservedTimestamp // // A transaction collects observed timestamps as it visits nodes in the cluster // when performing reads and writes. -var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp +var D6 = (&roachpb.Transaction{}).UpdateObservedTimestamp -// D6 ———————————————————————————————————————————————— +// D7 ———————————————————————————————————————————————— // // Transaction.ObservedTimestamps // @@ -91,7 +150,10 @@ var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp // purpose of this list is to avoid uncertainty related restarts which occur // when reading a value in the near future, per the global_uncertainty_limit // field. The list helps avoid these restarts by establishing a lower -// local_uncertainty_limit when evaluating a request on a node in the list. +// local_uncertainty_limit when evaluating a request on a node in the list. The +// local_uncertainty_limit can then be compared against the value's local +// timestamp to determine whether the value was written before or after the +// clock observation by the reader. // // Meaning // @@ -115,29 +177,32 @@ var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp // // Thus, expressed properly, we can say that when a node has been read from // successfully before by a transaction, uncertainty for values written by a -// leaseholder on that node is restricted to values with timestamps in the +// leaseholder on that node is restricted to values with local timestamps in the // interval [read_timestamp, first_visit_timestamp). An upper bound can be // placed on the uncertainty window because we are guaranteed that at the time // that the transaction first visited the node, none of the Ranges that it was a -// leaseholder for had served any writes at higher timestamps than the clock -// reading we observe. This implies the following property: +// leaseholder for had assigned local timestamps to any writes at higher +// timestamps than the clock reading we observe. This implies the following +// property: // // Any writes that the transaction may later see written by leaseholders on -// this node at higher timestamps than the observed timestamp could not have -// taken place causally before this transaction and can be ignored for the -// purposes of uncertainty. +// this node at higher local timestamps than the observed timestamp could not +// have taken place causally before this transaction and can be ignored for +// the purposes of uncertainty. // // There are two invariants necessary for this property to hold: // -// 1. a leaseholder's clock must always be equal to or greater than the timestamp -// of all writes that it has served. This is trivial to enforce for +// 1. a leaseholder's clock must always be equal to or greater than the local +// timestamp of all writes that it has served. This is trivial to enforce for // non-transactional writes. It is more complicated for transactional writes // which may move their commit timestamp forward over their lifetime before // committing, even after writing intents on remote Ranges. To accommodate -// this situation, transactions ensure that at the time of their commit, any -// leaseholder for a Range that contains one of its intent has an HLC clock -// with an equal or greater timestamp than the transaction's commit timestamp. -// TODO(nvanbenschoten): This is violated by txn refreshes. See #36431. +// this situation, MVCC key-values track their local timestamp separately +// from their version timestamp. This provides the flexibility to advance a +// key-value's version timestamp without advancing its local timestamp and +// ensure that at the time of a transaction's commit, any leaseholder for a +// Range that contains one of its intent has an HLC clock with a greater +// timestamp than that intent's local timestamp. // // 2. a leaseholder's clock must always be equal to or greater than the timestamp // of all writes that previous leaseholders for its Range have served. We @@ -153,14 +218,17 @@ var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp // // The property ensures that when this list holds a corresponding entry for the // node who owns the lease that the current request is executing under, we can -// run the request with the list's timestamp as the upper bound for its -// uncertainty interval, limiting (and often avoiding) uncertainty restarts. We -// do this by establishing a separate local_uncertainty_limit, which is set to -// the minimum of the global_uncertainty_limit and the node's observed timestamp -// entry in ComputeLocalUncertaintyLimit. +// compare the list's timestamp against the local timestamp of values in the +// range. This comparison establishes a total ordering across clock observations +// (local timestamp assignment for writers and observed timestamps for readers) +// and establish a partial ordering between writer and reader transactions. This +// partial ordering can limit (and often avoid) uncertainty restarts. We do this +// by establishing a separate local_uncertainty_limit for readers, which is set +// to the minimum of the global_uncertainty_limit and the node's observed +// timestamp entry in ComputeLocalUncertaintyLimit. // // However, as stated, the correctness property only holds for values at higher -// timestamps than the observed timestamp written *by leaseholders on this +// local timestamps than the observed timestamp written *by leaseholders on this // node*. This is critical, as the property tells us nothing about values // written by leaseholders on different nodes, even if a lease for one of those // Ranges has since moved to a node that we have an observed timestamp entry @@ -182,17 +250,18 @@ var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp // meaning for the purpose of reducing the transaction's uncertainty interval. // This is because there is no guarantee that at the time of acquiring the // observed timestamp from the follower node, the leaseholder hadn't already -// served writes at higher timestamps than the follower node's clock reflected. +// served writes with local timestamps higher than the follower node's clock +// reflected. // // However, if the transaction performing a follower read happens to have an // observed timestamp from the current leaseholder, this timestamp can be used // to reduce the transaction's uncertainty interval. Even though the read is // being served from a different replica in the range, the observed timestamp -// still places a bound on the values in the range that may have been written -// before the transaction began. -var D6 = roachpb.Transaction{}.ObservedTimestamps +// still places a bound on the local timestamps associated with values in the +// range that may have been written before the transaction began. +var D7 = roachpb.Transaction{}.ObservedTimestamps -// D7 ———————————————————————————————————————————————— +// D8 ———————————————————————————————————————————————— // // TimestampFromServerClock // @@ -219,7 +288,7 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // receive an uncertainty error. // // It is somewhat non-intuitive that non-transactional requests need uncertainty -// intervals — after all, they receive their timestamp to the leaseholder of the +// intervals — after all, they receive their timestamp on the leaseholder of the // only range that they talk to, so isn't every value with a commit timestamp // above their read timestamp certainly concurrent? The answer is surprisingly // "no" for the following reasons, so they cannot forgo the use of uncertainty @@ -229,7 +298,7 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // This means that there are times when the replica is not the leaseholder at // the point of timestamp allocation, and only becomes the leaseholder later. // In such cases, the timestamp assigned to the request is not guaranteed to -// be greater than the written_timestamp of all writes served by the range at +// be greater than the local timestamp of all writes served by the range at // the time of allocation. This is true despite invariants 1 & 2 from above, // because the replica allocating the timestamp is not yet the leaseholder. // @@ -253,7 +322,7 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // // 2. even if the replica's lease is stable and the timestamp is assigned to the // non-transactional request by the leaseholder, the assigned clock reading -// only reflects the written_timestamp of all of the writes served by the +// only reflects the local timestamp of all of the writes served by the // leaseholder (and previous leaseholders) thus far. This clock reading is // not guaranteed to lead the commit timestamp of all of these writes, // especially if they are committed remotely and resolved after the request @@ -266,21 +335,21 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // the leaseholder to be uncertain. Concretely, the non-transactional request // needs to consider values of the following form to be uncertain: // -// written_timestamp < local_limit && commit_timestamp < global_limit +// local_timestamp < local_limit && commit_timestamp < global_limit // // The value that the non-transactional request is observing may have been // written on the local leaseholder at time 10, its transaction may have been // committed remotely at time 20, acknowledged, then the non-transactional // request may have begun and received a timestamp of 15 from the local // leaseholder, then finally the value may have been resolved asynchronously -// and moved to timestamp 20 (written_timestamp: 10, commit_timestamp: 20). -// The failure of the non-transactional request to observe this value would -// be a stale read. +// and moved to timestamp 20 (local_timestamp: 10, commit_timestamp: 20). The +// failure of the non-transactional request to observe this value would be a +// stale read. // // For example, consider the following series of events: // - client begins a txn and is assigned provisional commit timestamp = 10 // - client's txn performs a Put(k, v1) -// - leaseholder serves Put(k, v1), lays down intent at written_timestamp = 10 +// - leaseholder serves Put(k, v1), lays down intent at local_timestamp = 10 // - client's txn performs a write elsewhere and hits a WriteTooOldError // that bumps its provisional commit timestamp to 20 // - client's txn refreshes to ts = 20. This notably happens without @@ -296,22 +365,15 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // server-side retry, bumps its read timestamp, and returns k = v1. Failure // to do so would be a stale read. // -// TODO(nvanbenschoten): expand on this when we fix #36431. For now, this can -// be framed in relation to synthetic timestamps, but it's easier to discuss -// in terms of the impending "written_timestamp" attribute of each value, -// even though written_timestamps do not yet exist in code. -// -// TODO(nvanbenschoten): add more direct testing for this when we fix #36431. -// // TODO(nvanbenschoten): add another reason here once we address #73292. // // Convenient, because non-transactional requests are always scoped to a // single-range, those that hit uncertainty errors can always retry on the // server, so these errors never bubble up to the client that initiated the // request. -var D7 = roachpb.Header{}.TimestampFromServerClock +var D8 = roachpb.Header{}.TimestampFromServerClock -// D8 ———————————————————————————————————————————————— +// D9 ———————————————————————————————————————————————— // // ComputeInterval // @@ -321,7 +383,7 @@ var D7 = roachpb.Header{}.TimestampFromServerClock // Similarly, observed timestamps can also assist a transaction even on its // first visit to a node in cases where it gets stuck waiting on locks for long // periods of time. -var D8 = ComputeInterval +var D9 = ComputeInterval // Ignore unused warnings. -var _, _, _, _, _, _, _, _ = D1, D2, D3, D4, D5, D6, D7, D8 +var _, _, _, _, _, _, _, _, _, _ = D0, D1, D2, D3, D4, D5, D6, D7, D8, D9 diff --git a/pkg/kv/kvserver/uncertainty/interval.go b/pkg/kv/kvserver/uncertainty/interval.go index be9de5d8e4f3..22c4e5520f78 100644 --- a/pkg/kv/kvserver/uncertainty/interval.go +++ b/pkg/kv/kvserver/uncertainty/interval.go @@ -31,11 +31,9 @@ import "github.com/cockroachdb/cockroach/pkg/util/hlc" // a range. This can lead to values that would otherwise be considered uncertain // by the original global limit to be considered "certainly concurrent", and // thus not causally related, with the transaction due to observed timestamps. -// -// However, the local limit does not apply to all committed values on a range. -// Specifically, values with "synthetic timestamps" must use the interval's -// global limit for the purposes of uncertainty, because observed timestamps do -// not apply to values with synthetic timestamps. +// However, the local limit does not operate on a value's version timestamp. It +// instead applies to a value's local timestamp, which is a recording of the +// local HLC clock on the leaseholder that originally wrote the value. // // Uncertainty intervals also apply to non-transactional requests that require // strong consistency (single-key linearizability). These requests defer their @@ -50,12 +48,15 @@ type Interval struct { LocalLimit hlc.ClockTimestamp } -// IsUncertain determines whether a value with the provided timestamp is -// uncertain to a reader with a ReadTimestamp below the value's and with -// the specified uncertainty interval. -func (in Interval) IsUncertain(valueTs hlc.Timestamp) bool { - if !in.LocalLimit.IsEmpty() && !valueTs.Synthetic { - return valueTs.LessEq(in.LocalLimit.ToTimestamp()) +// IsUncertain determines whether a value with the provided version and local +// timestamps is uncertain to a reader with a ReadTimestamp below the value's +// version timestamp and with the specified uncertainty interval. +func (in Interval) IsUncertain(valueTs hlc.Timestamp, localTs hlc.ClockTimestamp) bool { + if !in.LocalLimit.IsEmpty() && in.LocalLimit.Less(localTs) { + // The reader has an observed timestamp that precedes the local timestamp of + // this value. There is no uncertainty as the reader transaction must have + // started before the writer transaction completed, so they are concurrent. + return false } return valueTs.LessEq(in.GlobalLimit) } diff --git a/pkg/kv/kvserver/uncertainty/interval_test.go b/pkg/kv/kvserver/uncertainty/interval_test.go index 028640128944..065e2789522e 100644 --- a/pkg/kv/kvserver/uncertainty/interval_test.go +++ b/pkg/kv/kvserver/uncertainty/interval_test.go @@ -24,53 +24,83 @@ func TestInterval_IsUncertain(t *testing.T) { makeTs := func(walltime int64) hlc.Timestamp { return hlc.Timestamp{WallTime: walltime} } - makeSynTs := func(walltime int64) hlc.Timestamp { - return makeTs(walltime).WithSynthetic(true) - } emptyTs := makeTs(0) testCases := []struct { - localLim, globalLim, valueTs hlc.Timestamp - exp bool + localLim, globalLim, valueTs, localTs hlc.Timestamp + exp bool }{ - // Without synthetic value. - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(5), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(10), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), exp: false}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(5), exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(10), exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), exp: false}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), exp: false}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), exp: false}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(5), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(10), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), exp: false}, - // With synthetic value. - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeSynTs(5), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeSynTs(10), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeSynTs(15), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeSynTs(20), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeSynTs(25), exp: false}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeSynTs(5), exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeSynTs(10), exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeSynTs(15), exp: true}, // different - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeSynTs(20), exp: true}, // different - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeSynTs(25), exp: false}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeSynTs(5), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeSynTs(10), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeSynTs(15), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeSynTs(20), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeSynTs(25), exp: false}, + // Without local timestamp. localLim is ignored. + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(5), localTs: emptyTs, exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(10), localTs: emptyTs, exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), localTs: emptyTs, exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: emptyTs, exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: emptyTs, exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(5), localTs: emptyTs, exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(10), localTs: emptyTs, exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), localTs: emptyTs, exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: emptyTs, exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: emptyTs, exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(5), localTs: emptyTs, exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(10), localTs: emptyTs, exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), localTs: emptyTs, exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: emptyTs, exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: emptyTs, exp: false}, + // With local timestamp equal to value timestamp. + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(5), localTs: makeTs(5), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(10), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(15), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(20), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(25), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(5), localTs: makeTs(5), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(10), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(15), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(20), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(25), exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(5), localTs: makeTs(5), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(10), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(15), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(20), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(25), exp: false}, + // With local timestamp below value timestamp. + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(5), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(5), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(10), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(5), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(10), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(15), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(5), exp: false}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(10), exp: false}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(15), exp: false}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(20), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(5), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(5), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(10), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(5), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(10), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(15), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(5), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(10), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(15), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(20), exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(5), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(5), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(10), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(5), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(10), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(15), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(5), exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(10), exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(15), exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(20), exp: false}, // Empty uncertainty intervals. - {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(5), exp: false}, - {localLim: emptyTs, globalLim: emptyTs, valueTs: makeSynTs(5), exp: false}, + {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(10), localTs: emptyTs, exp: false}, + {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(10), localTs: makeTs(10), exp: false}, + {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(10), localTs: makeTs(5), exp: false}, } for _, test := range testCases { in := Interval{GlobalLimit: test.globalLim, LocalLimit: hlc.ClockTimestamp(test.localLim)} - require.Equal(t, test.exp, in.IsUncertain(test.valueTs), "%+v", test) + res := in.IsUncertain(test.valueTs, hlc.ClockTimestamp(test.localTs)) + require.Equal(t, test.exp, res, "%+v", test) } } diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index fc6b94dafd37..e6814bd9b86d 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -929,6 +929,7 @@ func MakeTransaction( // occurred, i.e. the maximum of ReadTimestamp and LastHeartbeat. func (t Transaction) LastActive() hlc.Timestamp { ts := t.LastHeartbeat + // TODO(nvanbenschoten): remove this when we remove synthetic timestamps. if !t.ReadTimestamp.Synthetic { ts.Forward(t.ReadTimestamp) } diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index f838acca0e97..46d84c2acf74 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -67,6 +67,12 @@ enum ValueType { // TIMESERIES is applied to values which contain InternalTimeSeriesData. TIMESERIES = 100; + + // MVCC_EXTENDED_ENCODING_SENTINEL is a sentinel tag value used in the + // MVCCValue extended encoding to indicate that an encoded Value is wrapped + // with a header. It should not be used in the Value encoding directly. See + // MVCCValue for more details. + MVCC_EXTENDED_ENCODING_SENTINEL = 101; } // Value specifies the value at a key. Multiple values at the same key are diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index a5312ed09203..640e91221fd4 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -934,6 +934,9 @@ var _ transactionRestartError = &WriteTooOldError{} // NewReadWithinUncertaintyIntervalError creates a new uncertainty retry error. // The read and existing timestamps as well as the txn are purely informational // and used for formatting the error message. +// TODO(nvanbenschoten): change localUncertaintyLimit to hlc.ClockTimestamp. +// TODO(nvanbenschoten): change existingTS to versionTs. +// TODO(nvanbenschoten): add localTs and include in error string. func NewReadWithinUncertaintyIntervalError( readTS, existingTS, localUncertaintyLimit hlc.Timestamp, txn *Transaction, ) *ReadWithinUncertaintyIntervalError { diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index 520d26de582f..385821fd2f20 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -1169,7 +1169,10 @@ func TestAssertEnginesEmpty(t *testing.T) { Key: []byte{0xde, 0xad, 0xbe, 0xef}, Timestamp: hlc.Timestamp{WallTime: 100}, } - require.NoError(t, batch.PutMVCC(key, []byte("foo"))) + value := storage.MVCCValue{ + Value: roachpb.MakeValueFromString("foo"), + } + require.NoError(t, batch.PutMVCC(key, value)) require.NoError(t, batch.Commit(false)) require.Error(t, assertEnginesEmpty([]storage.Engine{eng})) } diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index a92b50a04867..af04e1c3b7b6 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "mvcc_incremental_iterator.go", "mvcc_key.go", "mvcc_logical_ops.go", + "mvcc_value.go", "open.go", "pebble.go", "pebble_batch.go", @@ -110,6 +111,7 @@ go_test( "mvcc_logical_ops_test.go", "mvcc_stats_test.go", "mvcc_test.go", + "mvcc_value_test.go", "pebble_file_registry_test.go", "pebble_mvcc_scanner_test.go", "pebble_test.go", diff --git a/pkg/storage/batch_test.go b/pkg/storage/batch_test.go index 43f9cff57c39..495bc83628cb 100644 --- a/pkg/storage/batch_test.go +++ b/pkg/storage/batch_test.go @@ -1120,7 +1120,7 @@ func TestDecodeKey(t *testing.T) { t.Fatalf("%+v", err) } } else { - if err := b.PutMVCC(test, nil); err != nil { + if err := b.PutMVCC(test, MVCCValue{}); err != nil { t.Fatalf("%+v", err) } } diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index 7c848a84dd8b..be4628e9c2a7 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -190,7 +190,6 @@ func setupKeysWithIntent( txnIDWithLatestVersion := adjustTxnID(numVersions) otherTxnWithLatestVersion := txnIDCount + 2 otherTxnUUID := uuid.FromUint128(uint128.FromInts(0, uint64(otherTxnWithLatestVersion))) - val := []byte("value") var rvLockUpdate roachpb.LockUpdate for i := 1; i <= numVersions; i++ { // Assign txn IDs in a deterministic way that will mimic the end result of @@ -236,7 +235,7 @@ func setupKeysWithIntent( } } } - value := roachpb.Value{RawBytes: val} + value := roachpb.MakeValueFromString("value") batch := eng.NewBatch() for j := 0; j < numIntentKeys; j++ { putTxn := &txn @@ -1521,7 +1520,9 @@ func runExportToSst( key = encoding.EncodeUint32Ascending(key, uint32(i)) for j := 0; j < numRevisions; j++ { - err := batch.PutMVCC(MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(j + 1), Logical: 0}}, []byte("foobar")) + mvccKey := MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(j + 1), Logical: 0}} + mvccValue := MVCCValue{Value: roachpb.MakeValueFromString("foobar")} + err := batch.PutMVCC(mvccKey, mvccValue) if err != nil { b.Fatal(err) } @@ -1564,10 +1565,11 @@ func (noopWriter) Write(p []byte) (int, error) { return len(p), nil } func runCheckSSTConflicts(b *testing.B, numEngineKeys, numVersions, numSstKeys int, overlap bool) { keyBuf := append(make([]byte, 0, 64), []byte("key-")...) - value := make([]byte, 128) - for i := range value { - value[i] = 'a' + valueBuf := make([]byte, 128) + for i := range valueBuf { + valueBuf[i] = 'a' } + value := MVCCValue{Value: roachpb.MakeValueFromBytes(valueBuf)} eng := setupMVCCInMemPebble(b, "") defer eng.Close() @@ -1577,7 +1579,7 @@ func runCheckSSTConflicts(b *testing.B, numEngineKeys, numVersions, numSstKeys i for j := 0; j < numVersions; j++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: int64(j + 1)} - require.NoError(b, batch.PutMVCC(MVCCKey{key, ts}, value)) + require.NoError(b, batch.PutMVCC(MVCCKey{Key: key, Timestamp: ts}, value)) } require.NoError(b, batch.Commit(false)) } @@ -1605,7 +1607,7 @@ func runCheckSSTConflicts(b *testing.B, numEngineKeys, numVersions, numSstKeys i sstEnd.Key = append([]byte(nil), mvccKey.Key...) sstEnd.Timestamp = mvccKey.Timestamp } - require.NoError(b, sstWriter.Put(mvccKey, value)) + require.NoError(b, sstWriter.PutMVCC(mvccKey, value)) } sstWriter.Close() diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 7b86246eaca2..fa52de4a8f8d 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -589,8 +589,15 @@ type Writer interface { // timestamp is non-empty (see {PutUnversioned,PutIntent} if the timestamp // is empty). // - // It is safe to modify the contents of the arguments after Put returns. - PutMVCC(key MVCCKey, value []byte) error + // It is safe to modify the contents of the arguments after PutMVCC returns. + PutMVCC(key MVCCKey, value MVCCValue) error + // PutRawMVCC is like PutMVCC, but it accepts an encoded MVCCValue. It + // can be used to avoid decoding and immediately re-encoding an MVCCValue, + // but should generally be avoided due to the lack of type safety. + // + // It is safe to modify the contents of the arguments after PutRawMVCC + // returns. + PutRawMVCC(key MVCCKey, value []byte) error // PutUnversioned sets the given key to the value provided. It is for use // with inline metadata (not intents) and other unversioned keys (like // Range-ID local keys). diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index cc8653983721..74949377dff9 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -611,7 +611,8 @@ func TestEngineTimeBound(t *testing.T) { for i, time := range times { s := fmt.Sprintf("%02d", i) key := MVCCKey{Key: roachpb.Key(s), Timestamp: time} - if err := engine.PutMVCC(key, []byte(s)); err != nil { + value := MVCCValue{Value: roachpb.MakeValueFromString(s)} + if err := engine.PutMVCC(key, value); err != nil { t.Fatal(err) } } diff --git a/pkg/storage/enginepb/mvcc.proto b/pkg/storage/enginepb/mvcc.proto index ffaf911e898e..6faa0c8bf355 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -58,7 +58,7 @@ message MVCCMetadata { // the IntentHistory. optional int32 sequence = 1 [(gogoproto.nullable) = false, (gogoproto.casttype) = "TxnSeq"]; // Value is the value written to the key as part of the transaction at - // the above Sequence. + // the above Sequence. Value uses the roachpb.Value encoding. optional bytes value = 2; } diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index a528febf764f..24147eac6cfe 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -140,6 +140,38 @@ message IgnoredSeqNumRange { int32 end = 2 [(gogoproto.casttype) = "TxnSeq"]; } +// MVCCValueHeader holds MVCC-level metadata for a versioned value. +// Used by storage.MVCCValue. +message MVCCValueHeader { + option (gogoproto.equal) = true; + + // The local clock timestamp records the value of the local HLC clock on the + // leaseholder when the key was originally written. It is used to make claims + // about the relative real time ordering of the key-value's writer and readers + // when comparing a reader's uncertainty interval (and observed timestamps) to + // the key-value. Ignoring edge cases, readers with an observed timestamp from + // the key-value's leaseholder that is greater than the local clock timestamp + // stored in the key cannot make claims about real time ordering and must + // consider it possible that the key-value's write occurred before the read + // began. However, readers with an observed timestamp from the key-value's + // leaseholder that is less than the clock timestamp can claim that the reader + // captured that observed timestamp before the key-value was written and + // therefore can consider the key-value's write to have been concurrent with + // the read. In doing so, the reader can avoid an uncertainty restart. For + // more, see pkg/kv/kvserver/observedts/doc.go. + // + // If the local timestamp is empty, it is assumed to be equal to the key's + // version timestamp. This property is used to avoid storing the local clock + // timestamp in the common case where the HLC clock on the leaseholder is + // greater than or equal to the version timestamp at the time that it is + // written. If is safe for the local clock timestamp to be rounded down, as + // this will simply lead to additional uncertainty restarts. However, it is + // not safe for the local clock timestamp to be rounded up, as this could lead + // to stale reads. + util.hlc.Timestamp local_timestamp = 1 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; +} + // MVCCStatsDelta is convertible to MVCCStats, but uses signed variable width // encodings for most fields that make it more efficient to store negative // values. This makes the encodings incompatible. diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go index cd586706b286..6d28cbe95e01 100644 --- a/pkg/storage/intent_interleaving_iter_test.go +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -168,8 +168,18 @@ func checkAndOutputIter(iter MVCCIterator, b *strings.Builder) { } return } + mvccVal, err := DecodeMVCCValue(v1) + if err != nil { + fmt.Fprintf(b, "output: value decoding: %s\n", err) + return + } + mvccValBytes, err := mvccVal.Value.GetBytes() + if err != nil { + fmt.Fprintf(b, "output: value decoding: %s\n", err) + return + } fmt.Fprintf(b, "output: value k=%s ts=%s v=%s\n", - string(k1.Key), k1.Timestamp, string(v1)) + string(k1.Key), k1.Timestamp, string(mvccValBytes)) } // TestIntentInterleavingIter is a datadriven test consisting of two commands: @@ -299,7 +309,8 @@ func TestIntentInterleavingIter(t *testing.T) { var value string d.ScanArgs(t, "v", &value) mvccKey := MVCCKey{Key: key, Timestamp: ts} - if err := batch.PutMVCC(mvccKey, []byte(value)); err != nil { + mvccValue := MVCCValue{Value: roachpb.MakeValueFromString(value)} + if err := batch.PutMVCC(mvccKey, mvccValue); err != nil { return err.Error() } } @@ -567,7 +578,7 @@ func writeRandomData( if kv.Key.Timestamp.IsEmpty() { panic("timestamp should not be empty") } else { - require.NoError(t, batch.PutMVCC(kv.Key, kv.Value)) + require.NoError(t, batch.PutRawMVCC(kv.Key, kv.Value)) } } require.NoError(t, batch.Commit(true)) @@ -785,7 +796,9 @@ func writeBenchData( } for j := versionsPerKey; j >= 1; j-- { require.NoError(b, batch.PutMVCC( - MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(j)}}, []byte("value"))) + MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(j)}}, + MVCCValue{Value: roachpb.MakeValueFromString("value")}, + )) } } require.NoError(b, batch.Commit(true)) diff --git a/pkg/storage/multi_iterator_test.go b/pkg/storage/multi_iterator_test.go index eb71607ad801..aa84e9acb46e 100644 --- a/pkg/storage/multi_iterator_test.go +++ b/pkg/storage/multi_iterator_test.go @@ -108,7 +108,7 @@ func TestMultiIterator(t *testing.T) { t.Fatalf("%+v", err) } } else { - if err := batch.PutMVCC(MVCCKey{Key: k, Timestamp: ts}, v); err != nil { + if err := batch.PutRawMVCC(MVCCKey{Key: k, Timestamp: ts}, v); err != nil { t.Fatalf("%+v", err) } } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index c7c5ac51fbf6..252a14f9805a 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -95,7 +95,9 @@ func emptyKeyError() error { // MVCCKeyValue contains the raw bytes of the value for a key. type MVCCKeyValue struct { - Key MVCCKey + Key MVCCKey + // if Key.IsValue(), Value is an encoded MVCCValue. + // else, Value is an encoded MVCCMetadata. Value []byte } @@ -224,6 +226,7 @@ func updateStatsOnMerge(key roachpb.Key, valSize, nowNanos int64) enginepb.MVCCS // If this value is an intent, updates the intent counters. func updateStatsOnPut( key roachpb.Key, + prevIsValue bool, prevValSize int64, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize int64, orig, meta *enginepb.MVCCMetadata, @@ -317,8 +320,6 @@ func updateStatsOnPut( // Note that when meta.Timestamp equals orig.Timestamp, the computation is // moot, which is something our callers may exploit (since retrieving the // previous version is not for free). - prevIsValue := prevValSize > 0 - if prevIsValue { // If the previous value (exists and) was not a deletion tombstone, make it // live at orig.Timestamp. We don't have to do anything if there is a @@ -385,6 +386,7 @@ func updateStatsOnPut( // counters if commit=true. func updateStatsOnResolve( key roachpb.Key, + prevIsValue bool, prevValSize int64, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize int64, orig, meta *enginepb.MVCCMetadata, @@ -394,7 +396,8 @@ func updateStatsOnResolve( if isSysLocal(key) { // Straightforward: old contribution goes, new contribution comes, and we're done. - ms.SysBytes += (metaKeySize + metaValSize) - (origMetaValSize + origMetaKeySize) + ms.SysBytes -= origMetaKeySize + origMetaValSize + orig.KeyBytes + orig.ValBytes + ms.SysBytes += metaKeySize + metaValSize + meta.KeyBytes + meta.ValBytes return ms } @@ -447,7 +450,6 @@ func updateStatsOnResolve( // and so in that case the caller may simply pass prevValSize=0 and can // skip computing that quantity in the first place. _ = updateStatsOnPut - prevIsValue := prevValSize > 0 if prevIsValue { ms.LiveBytes += MVCCVersionTimestampSize + prevValSize @@ -857,7 +859,7 @@ func mvccGetMetadata( if !iterAlreadyPositioned { iter.SeekGE(metaKey) } - if ok, err := iter.Valid(); !ok { + if ok, err = iter.Valid(); !ok { return false, 0, 0, err } @@ -874,13 +876,22 @@ func mvccGetMetadata( int64(len(iter.UnsafeValue())), nil } + unsafeValRaw := iter.UnsafeValue() + unsafeVal, ok, err := tryDecodeSimpleMVCCValue(unsafeValRaw) + if !ok && err == nil { + unsafeVal, err = decodeExtendedMVCCValue(unsafeValRaw) + } + if err != nil { + return false, 0, 0, err + } + meta.Reset() // For values, the size of keys is always accounted for as // MVCCVersionTimestampSize. The size of the metadata key is // accounted for separately. meta.KeyBytes = MVCCVersionTimestampSize - meta.ValBytes = int64(len(iter.UnsafeValue())) - meta.Deleted = meta.ValBytes == 0 + meta.ValBytes = int64(len(unsafeValRaw)) + meta.Deleted = unsafeVal.IsTombstone() meta.Timestamp = unsafeKey.Timestamp.ToLegacyTimestamp() return true, int64(unsafeKey.EncodedSize()) - meta.KeyBytes, 0, nil } @@ -892,8 +903,8 @@ func mvccGetMetadata( type putBuffer struct { meta enginepb.MVCCMetadata newMeta enginepb.MVCCMetadata - ts hlc.LegacyTimestamp - tmpbuf []byte + ts hlc.LegacyTimestamp // avoids heap allocations + tmpbuf []byte // avoids heap allocations } var putBufferPool = sync.Pool{ @@ -1064,19 +1075,11 @@ func mvccPutUsingIter( localTimestamp hlc.ClockTimestamp, value roachpb.Value, txn *roachpb.Transaction, - valueFn func(optionalValue) ([]byte, error), + valueFn func(optionalValue) (roachpb.Value, error), ) error { - var rawBytes []byte - if valueFn == nil { - if !value.Timestamp.IsEmpty() { - return errors.Errorf("cannot have timestamp set in value on Put") - } - rawBytes = value.RawBytes - } - buf := newPutBuffer() - err := mvccPutInternal(ctx, writer, iter, ms, key, timestamp, localTimestamp, rawBytes, txn, buf, valueFn) + err := mvccPutInternal(ctx, writer, iter, ms, key, timestamp, localTimestamp, value, txn, buf, valueFn) // Using defer would be more convenient, but it is measurably slower. buf.release() @@ -1094,11 +1097,11 @@ func maybeGetValue( ctx context.Context, iter MVCCIterator, key roachpb.Key, - value []byte, + value roachpb.Value, exists bool, readTimestamp hlc.Timestamp, - valueFn func(optionalValue) ([]byte, error), -) ([]byte, error) { + valueFn func(optionalValue) (roachpb.Value, error), +) (roachpb.Value, error) { // If a valueFn is specified, read existing value using the iter. if valueFn == nil { return value, nil @@ -1108,7 +1111,7 @@ func maybeGetValue( var err error exVal, _, err = mvccGet(ctx, iter, key, readTimestamp, MVCCGetOptions{Tombstones: true}) if err != nil { - return nil, err + return roachpb.Value{}, err } } return valueFn(exVal) @@ -1164,9 +1167,9 @@ func replayTransactionalWrite( meta *enginepb.MVCCMetadata, key roachpb.Key, timestamp hlc.Timestamp, - value []byte, + value roachpb.Value, txn *roachpb.Transaction, - valueFn func(optionalValue) ([]byte, error), + valueFn func(optionalValue) (roachpb.Value, error), ) error { var found bool var writtenValue []byte @@ -1236,9 +1239,9 @@ func replayTransactionalWrite( // To ensure the transaction is idempotent, we must assert that the // calculated value on this replay is the same as the one we've previously // written. - if !bytes.Equal(value, writtenValue) { + if !bytes.Equal(value.RawBytes, writtenValue) { return errors.AssertionFailedf("transaction %s with sequence %d has a different value %+v after recomputing from what was written: %+v", - txn.ID, txn.Sequence, value, writtenValue) + txn.ID, txn.Sequence, value.RawBytes, writtenValue) } return nil } @@ -1284,18 +1287,20 @@ func mvccPutInternal( key roachpb.Key, timestamp hlc.Timestamp, localTimestamp hlc.ClockTimestamp, - value []byte, + value roachpb.Value, txn *roachpb.Transaction, buf *putBuffer, - valueFn func(optionalValue) ([]byte, error), + valueFn func(optionalValue) (roachpb.Value, error), ) error { if len(key) == 0 { return emptyKeyError() } - if timestamp.WallTime < 0 { return errors.Errorf("cannot write to %q at timestamp %s", key, timestamp) } + if !value.Timestamp.IsEmpty() { + return errors.Errorf("cannot have timestamp set in value") + } metaKey := MakeMVCCMetadataKey(key) ok, origMetaKeySize, origMetaValSize, err := @@ -1320,10 +1325,10 @@ func mvccPutInternal( if value, err = maybeGetValue(ctx, iter, key, value, ok, timestamp, valueFn); err != nil { return err } - if value == nil { + if !value.IsPresent() { metaKeySize, metaValSize, err = 0, 0, writer.ClearUnversioned(metaKey.Key) } else { - buf.meta = enginepb.MVCCMetadata{RawBytes: value} + buf.meta = enginepb.MVCCMetadata{RawBytes: value.RawBytes} metaKeySize, metaValSize, err = buf.putInlineMeta(writer, metaKey, &buf.meta) } if ms != nil { @@ -1370,6 +1375,7 @@ func mvccPutInternal( } var maybeTooOldErr error + var prevIsValue bool var prevValSize int64 if ok { // There is existing metadata for this key; ensure our write is permitted. @@ -1466,24 +1472,32 @@ func mvccPutInternal( // delete the old intent, taking care with MVCC stats. logicalOp = MVCCUpdateIntentOpType if metaTimestamp.Less(writeTimestamp) { + versionKey := metaKey + versionKey.Timestamp = metaTimestamp + { // If the older write intent has a version underneath it, we need to // read its size because its GCBytesAge contribution may change as we // move the intent above it. A similar phenomenon occurs in // MVCCResolveWriteIntent. - latestKey := MVCCKey{Key: key, Timestamp: metaTimestamp} - _, prevUnsafeVal, haveNextVersion, err := unsafeNextVersion(iter, latestKey) + _, prevUnsafeVal, haveNextVersion, err := unsafeNextVersion(iter, versionKey) if err != nil { return err } if haveNextVersion { + prevVal, ok, err := tryDecodeSimpleMVCCValue(prevUnsafeVal) + if !ok && err == nil { + prevVal, err = decodeExtendedMVCCValue(prevUnsafeVal) + } + if err != nil { + return err + } + prevIsValue = prevVal.Value.IsPresent() prevValSize = int64(len(prevUnsafeVal)) } iter = nil // prevent accidental use below } - versionKey := metaKey - versionKey.Timestamp = metaTimestamp if err := writer.ClearMVCC(versionKey); err != nil { return err } @@ -1575,30 +1589,39 @@ func mvccPutInternal( } } + versionKey := metaKey + versionKey.Timestamp = writeTimestamp + + versionValue := MVCCValue{} + versionValue.Value = value + versionValue.LocalTimestamp = localTimestamp + if !versionValue.LocalTimestampNeeded(versionKey) { + versionValue.LocalTimestamp = hlc.ClockTimestamp{} + } + + // Write the mvcc metadata now that we have sizes for the latest + // versioned value. For values, the size of keys is always accounted + // for as MVCCVersionTimestampSize. The size of the metadata key is + // accounted for separately. + newMeta := &buf.newMeta { var txnMeta *enginepb.TxnMeta if txn != nil { txnMeta = &txn.TxnMeta // If we bumped the WriteTimestamp, we update both the TxnMeta and the // MVCCMetadata.Timestamp. - if txnMeta.WriteTimestamp != writeTimestamp { + if txnMeta.WriteTimestamp != versionKey.Timestamp { txnMetaCpy := *txnMeta - txnMetaCpy.WriteTimestamp.Forward(writeTimestamp) + txnMetaCpy.WriteTimestamp.Forward(versionKey.Timestamp) txnMeta = &txnMetaCpy } } - buf.newMeta.Txn = txnMeta - buf.newMeta.Timestamp = writeTimestamp.ToLegacyTimestamp() + newMeta.Txn = txnMeta } - newMeta := &buf.newMeta - - // Write the mvcc metadata now that we have sizes for the latest - // versioned value. For values, the size of keys is always accounted - // for as MVCCVersionTimestampSize. The size of the metadata key is - // accounted for separately. + newMeta.Timestamp = versionKey.Timestamp.ToLegacyTimestamp() newMeta.KeyBytes = MVCCVersionTimestampSize - newMeta.ValBytes = int64(len(value)) - newMeta.Deleted = value == nil + newMeta.ValBytes = int64(encodedMVCCValueSize(versionValue)) + newMeta.Deleted = versionValue.IsTombstone() var metaKeySize, metaValSize int64 if newMeta.Txn != nil { @@ -1629,15 +1652,13 @@ func mvccPutInternal( // that the meta key is always ordered before the value key and that // RocksDB's skiplist memtable implementation includes a fast-path for // sequential insertion patterns. - versionKey := metaKey - versionKey.Timestamp = writeTimestamp - if err := writer.PutMVCC(versionKey, value); err != nil { + if err := writer.PutMVCC(versionKey, versionValue); err != nil { return err } // Update MVCC stats. if ms != nil { - ms.Add(updateStatsOnPut(key, prevValSize, origMetaKeySize, origMetaValSize, + ms.Add(updateStatsOnPut(key, prevIsValue, prevValSize, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize, meta, newMeta)) } @@ -1680,11 +1701,11 @@ func MVCCIncrement( var int64Val int64 var newInt64Val int64 - valueFn := func(value optionalValue) ([]byte, error) { + valueFn := func(value optionalValue) (roachpb.Value, error) { if value.IsPresent() { var err error if int64Val, err = value.GetInt(); err != nil { - return nil, errors.Errorf("key %q does not contain an integer value", key) + return roachpb.Value{}, errors.Errorf("key %q does not contain an integer value", key) } } @@ -1692,7 +1713,7 @@ func MVCCIncrement( if willOverflow(int64Val, inc) { // Return the old value, since we've failed to modify it. newInt64Val = int64Val - return nil, &roachpb.IntegerOverflowError{ + return roachpb.Value{}, &roachpb.IntegerOverflowError{ Key: key, CurrentValue: int64Val, IncrementValue: inc, @@ -1703,7 +1724,7 @@ func MVCCIncrement( newValue := roachpb.Value{} newValue.SetInt(newInt64Val) newValue.InitChecksum(key) - return newValue.RawBytes, nil + return newValue, nil } err := mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) @@ -1794,19 +1815,19 @@ func mvccConditionalPutUsingIter( allowNoExisting CPutMissingBehavior, txn *roachpb.Transaction, ) error { - valueFn := func(existVal optionalValue) ([]byte, error) { + valueFn := func(existVal optionalValue) (roachpb.Value, error) { if expValPresent, existValPresent := len(expBytes) != 0, existVal.IsPresent(); expValPresent && existValPresent { if !bytes.Equal(expBytes, existVal.TagAndDataBytes()) { - return nil, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &roachpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } } else if expValPresent != existValPresent && (existValPresent || !bool(allowNoExisting)) { - return nil, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &roachpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } - return value.RawBytes, nil + return value, nil } return mvccPutUsingIter(ctx, writer, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) } @@ -1871,20 +1892,20 @@ func mvccInitPutUsingIter( failOnTombstones bool, txn *roachpb.Transaction, ) error { - valueFn := func(existVal optionalValue) ([]byte, error) { + valueFn := func(existVal optionalValue) (roachpb.Value, error) { if failOnTombstones && existVal.IsTombstone() { // We found a tombstone and failOnTombstones is true: fail. - return nil, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &roachpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } if existVal.IsPresent() && !existVal.EqualTagAndData(value) { // The existing value does not match the supplied value. - return nil, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &roachpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } - return value.RawBytes, nil + return value, nil } return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) } @@ -2091,8 +2112,7 @@ func MVCCClearTimeRange( defer iter.Close() var clearedMetaKey MVCCKey - var clearedMeta enginepb.MVCCMetadata - var restoredMeta enginepb.MVCCMetadata + var clearedMeta, restoredMeta enginepb.MVCCMetadata iter.SeekGE(MVCCKey{Key: key}) for { if ok, err := iter.Valid(); err != nil { @@ -2102,16 +2122,20 @@ func MVCCClearTimeRange( } k := iter.UnsafeKey() + vRaw := iter.UnsafeValue() + v, err := DecodeMVCCValue(vRaw) + if err != nil { + return nil, err + } if len(clearedMetaKey.Key) > 0 { metaKeySize := int64(clearedMetaKey.EncodedSize()) if bytes.Equal(clearedMetaKey.Key, k.Key) { // Since the key matches, our previous clear "restored" this revision of // the this key, so update the stats with this as the "restored" key. - valueSize := int64(len(iter.Value())) restoredMeta.KeyBytes = MVCCVersionTimestampSize - restoredMeta.Deleted = valueSize == 0 - restoredMeta.ValBytes = valueSize + restoredMeta.ValBytes = int64(len(vRaw)) + restoredMeta.Deleted = v.IsTombstone() restoredMeta.Timestamp = k.Timestamp.ToLegacyTimestamp() ms.Add(updateStatsOnClear( @@ -2136,8 +2160,8 @@ func MVCCClearTimeRange( clearMatchingKey(k) clearedMetaKey.Key = append(clearedMetaKey.Key[:0], k.Key...) clearedMeta.KeyBytes = MVCCVersionTimestampSize - clearedMeta.ValBytes = int64(len(iter.UnsafeValue())) - clearedMeta.Deleted = clearedMeta.ValBytes == 0 + clearedMeta.ValBytes = int64(len(vRaw)) + clearedMeta.Deleted = v.IsTombstone() clearedMeta.Timestamp = k.Timestamp.ToLegacyTimestamp() // Move the iterator to the next key/value in linear iteration even if it @@ -2215,7 +2239,7 @@ func MVCCDeleteRange( var keys []roachpb.Key for i, kv := range res.KVs { if err := mvccPutInternal( - ctx, rw, iter, ms, kv.Key, timestamp, localTimestamp, nil, txn, buf, nil, + ctx, rw, iter, ms, kv.Key, timestamp, localTimestamp, noValue, txn, buf, nil, ); err != nil { return nil, nil, 0, err } @@ -3005,9 +3029,12 @@ func mvccResolveWriteIntent( // can be considered empty and marked for removal (removeIntent = true). // If only part of the intent history was rolled back, but the intent still // remains, the rolledBackVal is set to a non-nil value. - var rolledBackVal []byte + var rolledBackVal *MVCCValue if len(intent.IgnoredSeqNums) > 0 { // NOTE: mvccMaybeRewriteIntentHistory mutates its meta argument. + // TODO(nvanbenschoten): this is an awkward interface. We shouldn't + // be mutating meta and we shouldn't be restoring the previous value + // here. Instead, this should all be handled down below. var removeIntent bool removeIntent, rolledBackVal, err = mvccMaybeRewriteIntentHistory(ctx, rw, intent.IgnoredSeqNums, meta, latestKey) if err != nil { @@ -3051,6 +3078,9 @@ func mvccResolveWriteIntent( // is because removeIntent implies rolledBackVal == nil, pushed == false, and // commit == false. if commit || pushed || rolledBackVal != nil { + buf.newMeta = *meta + newMeta := &buf.newMeta + // The intent might be committing at a higher timestamp, or it might be // getting pushed. newTimestamp := intent.Txn.WriteTimestamp @@ -3063,35 +3093,13 @@ func mvccResolveWriteIntent( metaTimestamp, newTimestamp, commit, pushed, rolledBackVal != nil) } - buf.newMeta = *meta - // Set the timestamp for upcoming write (or at least the stats update). - buf.newMeta.Timestamp = newTimestamp.ToLegacyTimestamp() - buf.newMeta.Txn.WriteTimestamp = newTimestamp - - // Update or remove the metadata key. - var metaKeySize, metaValSize int64 - if !commit { - // Keep existing intent if we're updating it. We update the existing - // metadata's timestamp instead of using the supplied intent meta to avoid - // overwriting a newer epoch (see comments above). The pusher's job isn't - // to do anything to update the intent but to move the timestamp forward, - // even if it can. - metaKeySize, metaValSize, err = buf.putIntentMeta( - ctx, rw, metaKey, &buf.newMeta, true /* alreadyExists */) - } else { - metaKeySize = int64(metaKey.EncodedSize()) - err = rw.ClearIntent(metaKey.Key, canSingleDelHelper.onCommitIntent(), meta.Txn.ID) - } - if err != nil { - return false, err - } - - // If we're moving the intent's timestamp, adjust stats and - // rewrite it. + // If we're moving the intent's timestamp, rewrite it and adjust stats. + var prevIsValue bool var prevValSize int64 if timestampChanged { - oldKey := MVCCKey{Key: intent.Key, Timestamp: metaTimestamp} - newKey := MVCCKey{Key: intent.Key, Timestamp: newTimestamp} + oldKey := latestKey + newKey := oldKey + newKey.Timestamp = newTimestamp // Rewrite the versioned value at the new timestamp. iter.SeekGE(oldKey) @@ -3100,16 +3108,39 @@ func mvccResolveWriteIntent( } else if !valid || !iter.UnsafeKey().Equal(oldKey) { return false, errors.Errorf("existing intent value missing: %s", oldKey) } - value := iter.UnsafeValue() + oldValue, err := DecodeMVCCValue(iter.UnsafeValue()) + if err != nil { + return false, err + } // Special case: If mvccMaybeRewriteIntentHistory rolled back to a value // in the intent history and wrote that at oldKey, iter would not be able // to "see" the value since it was created before that value was written // to the engine. In this case, reuse the value returned by // mvccMaybeRewriteIntentHistory. if rolledBackVal != nil { - value = rolledBackVal + oldValue = *rolledBackVal + } + + // The local timestamp does not change during intent resolution unless the + // resolver provides a clock observation from this node that was captured + // while the transaction was still pending, in which case it can be advanced + // to the observed timestamp. + newValue := oldValue + newValue.LocalTimestamp = oldValue.GetLocalTimestamp(oldKey) + newValue.LocalTimestamp.Forward(intent.ClockWhilePending.Timestamp) + if !newValue.LocalTimestampNeeded(newKey) { + newValue.LocalTimestamp = hlc.ClockTimestamp{} } - if err = rw.PutMVCC(newKey, value); err != nil { + + // Update the MVCC metadata with the timestamp for the upcoming write (or + // at least the stats update). + newMeta.Txn.WriteTimestamp = newTimestamp + newMeta.Timestamp = newTimestamp.ToLegacyTimestamp() + newMeta.KeyBytes = MVCCVersionTimestampSize + newMeta.ValBytes = int64(encodedMVCCValueSize(newValue)) + newMeta.Deleted = newValue.IsTombstone() + + if err = rw.PutMVCC(newKey, newValue); err != nil { return false, err } if err = rw.ClearMVCC(oldKey); err != nil { @@ -3128,14 +3159,41 @@ func mvccResolveWriteIntent( if valid, err := iter.Valid(); err != nil { return false, err } else if valid && iter.UnsafeKey().Key.Equal(oldKey.Key) { + unsafeValRaw := iter.UnsafeValue() + prevVal, prevValOK, err := tryDecodeSimpleMVCCValue(unsafeValRaw) + if !prevValOK && err == nil { + prevVal, err = decodeExtendedMVCCValue(unsafeValRaw) + } + if err != nil { + return false, err + } + prevIsValue = prevVal.Value.IsPresent() prevValSize = int64(len(iter.UnsafeValue())) } } + // Update or remove the metadata key. + var metaKeySize, metaValSize int64 + if !commit { + // Keep existing intent if we're updating it. We update the existing + // metadata's timestamp instead of using the supplied intent meta to avoid + // overwriting a newer epoch (see comments above). The pusher's job isn't + // to do anything to update the intent but to move the timestamp forward, + // even if it can. + metaKeySize, metaValSize, err = buf.putIntentMeta( + ctx, rw, metaKey, newMeta, true /* alreadyExists */) + } else { + metaKeySize = int64(metaKey.EncodedSize()) + err = rw.ClearIntent(metaKey.Key, canSingleDelHelper.onCommitIntent(), meta.Txn.ID) + } + if err != nil { + return false, err + } + // Update stat counters related to resolving the intent. if ms != nil { - ms.Add(updateStatsOnResolve(intent.Key, prevValSize, origMetaKeySize, origMetaValSize, - metaKeySize, metaValSize, meta, &buf.newMeta, commit)) + ms.Add(updateStatsOnResolve(intent.Key, prevIsValue, prevValSize, origMetaKeySize, origMetaValSize, + metaKeySize, metaValSize, meta, newMeta, commit)) } // Log the logical MVCC operation. @@ -3178,7 +3236,7 @@ func mvccResolveWriteIntent( nextKey := latestKey.Next() ok = false var unsafeNextKey MVCCKey - var unsafeNextValue []byte + var unsafeNextValueRaw []byte if nextKey.IsValue() { // The latestKey was not the smallest possible timestamp {WallTime: 0, // Logical: 1}. Practically, this is the only case that will occur in @@ -3195,7 +3253,7 @@ func mvccResolveWriteIntent( // particular timestamp. return false, errors.Errorf("expected an MVCC value key: %s", unsafeNextKey) } - unsafeNextValue = iter.UnsafeValue() + unsafeNextValueRaw = iter.UnsafeValue() } else { ok = false } @@ -3217,12 +3275,15 @@ func mvccResolveWriteIntent( } // Get the bytes for the next version so we have size for stat counts. - valueSize := int64(len(unsafeNextValue)) + unsafeNextValue, err := DecodeMVCCValue(unsafeNextValueRaw) + if err != nil { + return false, err + } // Update the keyMetadata with the next version. buf.newMeta = enginepb.MVCCMetadata{ - Deleted: valueSize == 0, + Deleted: unsafeNextValue.IsTombstone(), KeyBytes: MVCCVersionTimestampSize, - ValBytes: valueSize, + ValBytes: int64(len(unsafeNextValueRaw)), } if err = rw.ClearIntent(metaKey.Key, canSingleDelHelper.onAbortIntent(), meta.Txn.ID); err != nil { return false, err @@ -3253,7 +3314,7 @@ func mvccMaybeRewriteIntentHistory( ignoredSeqNums []enginepb.IgnoredSeqNumRange, meta *enginepb.MVCCMetadata, latestKey MVCCKey, -) (remove bool, updatedVal []byte, err error) { +) (remove bool, updatedVal *MVCCValue, err error) { if !enginepb.TxnSeqIsIgnored(meta.Txn.Sequence, ignoredSeqNums) { // The latest write was not ignored. Nothing to do here. We'll // proceed with the intent as usual. @@ -3277,15 +3338,19 @@ func mvccMaybeRewriteIntentHistory( // Otherwise, we place back the write at that history entry // back into the intent. - restoredVal := meta.IntentHistory[i].Value + restoredValRaw := meta.IntentHistory[i].Value + restoredVal, err := DecodeMVCCValue(restoredValRaw) + if err != nil { + return false, nil, err + } meta.Txn.Sequence = meta.IntentHistory[i].Sequence meta.IntentHistory = meta.IntentHistory[:i] - meta.Deleted = len(restoredVal) == 0 - meta.ValBytes = int64(len(restoredVal)) + meta.Deleted = restoredVal.IsTombstone() + meta.ValBytes = int64(len(restoredValRaw)) // And also overwrite whatever was there in storage. err = engine.PutMVCC(latestKey, restoredVal) - return false, restoredVal, err + return false, &restoredVal, err } // IterAndBuf used to pass iterators and buffers between MVCC* calls, allowing @@ -3658,19 +3723,26 @@ func MVCCGarbageCollect( break } if ms != nil { - // FIXME: use prevNanos instead of unsafeIterKey.Timestamp, except - // when it's a deletion. - valSize := int64(len(iter.UnsafeValue())) + unsafeValRaw := iter.UnsafeValue() + unsafeVal, unsafeValOK, err := tryDecodeSimpleMVCCValue(unsafeValRaw) + if !unsafeValOK && err == nil { + unsafeVal, err = decodeExtendedMVCCValue(unsafeValRaw) + } + if err != nil { + return err + } + + keySize := MVCCVersionTimestampSize + valSize := int64(len(unsafeValRaw)) // A non-deletion becomes non-live when its newer neighbor shows up. // A deletion tombstone becomes non-live right when it is created. fromNS := prevNanos - if valSize == 0 { + if unsafeVal.IsTombstone() { fromNS = unsafeIterKey.Timestamp.WallTime } - ms.Add(updateStatsOnGC(gcKey.Key, MVCCVersionTimestampSize, - valSize, nil, fromNS)) + ms.Add(updateStatsOnGC(gcKey.Key, keySize, valSize, nil, fromNS)) } count++ if err := rw.ClearMVCC(unsafeIterKey); err != nil { @@ -3864,10 +3936,22 @@ func ComputeStatsForRange( if implicitMeta { // No MVCCMetadata entry for this series of keys. + var isTombstone bool + { + mvccValue, ok, err := tryDecodeSimpleMVCCValue(unsafeValue) + if !ok && err == nil { + mvccValue, err = decodeExtendedMVCCValue(unsafeValue) + } + if err != nil { + return ms, errors.Wrap(err, "unable to decode MVCCValue") + } + isTombstone = mvccValue.IsTombstone() + } + meta.Reset() meta.KeyBytes = MVCCVersionTimestampSize meta.ValBytes = int64(len(unsafeValue)) - meta.Deleted = len(unsafeValue) == 0 + meta.Deleted = isTombstone meta.Timestamp.WallTime = unsafeKey.Timestamp.WallTime } @@ -3941,7 +4025,18 @@ func ComputeStatsForRange( accrueGCAgeNanos = meta.Timestamp.WallTime } else { // Overwritten value. Is it a deletion tombstone? - isTombstone := len(unsafeValue) == 0 + var isTombstone bool + { + mvccValue, ok, err := tryDecodeSimpleMVCCValue(unsafeValue) + if !ok && err == nil { + mvccValue, err = decodeExtendedMVCCValue(unsafeValue) + } + if err != nil { + return ms, errors.Wrap(err, "unable to decode MVCCValue") + } + isTombstone = mvccValue.IsTombstone() + } + if isTombstone { // The contribution of the tombstone picks up GCByteAge from its own timestamp on. ms.GCBytesAge += totalBytes * (nowNanos/1e9 - unsafeKey.Timestamp.WallTime/1e9) diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 702040da32b2..5b5e71ae7fa9 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -52,7 +52,7 @@ import ( // txn_advance t= ts=[,] // txn_status t= status= // -// resolve_intent t= k= [status=] +// resolve_intent t= k= [status=] [clockWhilePending=[,]] // check_intent k= [none] // // cput [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] [cond=] @@ -124,7 +124,12 @@ func TestMVCCHistories(t *testing.T) { buf.Printf("meta: %v -> %+v\n", r.Key, &meta) } } else { - buf.Printf("data: %v -> %s\n", r.Key, roachpb.Value{RawBytes: r.Value}.PrettyPrint()) + val, err := DecodeMVCCValue(r.Value) + if err != nil { + buf.Printf("data: %v -> error decoding value %v: %v\n", r.Key, r.Value, err) + } else { + buf.Printf("data: %v -> %s\n", r.Key, val) + } } return nil }) @@ -543,14 +548,20 @@ func cmdResolveIntent(e *evalCtx) error { txn := e.getTxn(mandatory) key := e.getKey() status := e.getTxnStatus() - return e.resolveIntent(e.tryWrapForIntentPrinting(e.engine), key, txn, status) + clockWhilePending := hlc.ClockTimestamp(e.getTsWithName("clockWhilePending")) + return e.resolveIntent(e.tryWrapForIntentPrinting(e.engine), key, txn, status, clockWhilePending) } func (e *evalCtx) resolveIntent( - rw ReadWriter, key roachpb.Key, txn *roachpb.Transaction, resolveStatus roachpb.TransactionStatus, + rw ReadWriter, + key roachpb.Key, + txn *roachpb.Transaction, + resolveStatus roachpb.TransactionStatus, + clockWhilePending hlc.ClockTimestamp, ) error { intent := roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}) intent.Status = resolveStatus + intent.ClockWhilePending = roachpb.ObservedTimestamp{Timestamp: clockWhilePending} _, err := MVCCResolveWriteIntent(e.ctx, rw, nil, intent) return err } @@ -608,7 +619,7 @@ func cmdCPut(e *evalCtx) error { return err } if resolve { - return e.resolveIntent(rw, key, txn, resolveStatus) + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } return nil }) @@ -625,7 +636,7 @@ func cmdDelete(e *evalCtx) error { return err } if resolve { - return e.resolveIntent(rw, key, txn, resolveStatus) + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } return nil }) @@ -658,7 +669,7 @@ func cmdDeleteRange(e *evalCtx) error { } if resolve { - return e.resolveIntent(rw, key, txn, resolveStatus) + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } return nil }) @@ -726,7 +737,7 @@ func cmdIncrement(e *evalCtx) error { } e.results.buf.Printf("inc: current value = %d\n", curVal) if resolve { - return e.resolveIntent(rw, key, txn, resolveStatus) + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } return nil }) @@ -734,14 +745,7 @@ func cmdIncrement(e *evalCtx) error { func cmdMerge(e *evalCtx) error { key := e.getKey() - var value string - e.scanArg("v", &value) - var val roachpb.Value - if e.hasArg("raw") { - val.RawBytes = []byte(value) - } else { - val.SetString(value) - } + val := e.getVal() ts := e.getTs(nil) return e.withWriter("merge", func(rw ReadWriter) error { return MVCCMerge(e.ctx, rw, nil, key, ts, val) @@ -763,7 +767,7 @@ func cmdPut(e *evalCtx) error { return err } if resolve { - return e.resolveIntent(rw, key, txn, resolveStatus) + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } return nil }) diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index 28a2bf88aec6..79c1363b89f9 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -39,13 +39,11 @@ import ( const all, latest = true, false -func makeKVT(key roachpb.Key, value []byte, ts hlc.Timestamp) MVCCKeyValue { - return MVCCKeyValue{Key: MVCCKey{Key: key, Timestamp: ts}, Value: value} +func makeKVT(key roachpb.Key, value roachpb.Value, ts hlc.Timestamp) MVCCKeyValue { + return MVCCKeyValue{Key: MVCCKey{Key: key, Timestamp: ts}, Value: value.RawBytes} } -func makeKVTxn( - key roachpb.Key, val []byte, ts hlc.Timestamp, -) (roachpb.Transaction, roachpb.Value, roachpb.Intent) { +func makeKVTxn(key roachpb.Key, ts hlc.Timestamp) (roachpb.Transaction, roachpb.Intent) { txnID := uuid.MakeV4() txnMeta := enginepb.TxnMeta{ Key: key, @@ -53,12 +51,12 @@ func makeKVTxn( Epoch: 1, WriteTimestamp: ts, } - return roachpb.Transaction{ - TxnMeta: txnMeta, - ReadTimestamp: ts, - }, roachpb.Value{ - RawBytes: val, - }, roachpb.MakeIntent(&txnMeta, key) + txn := roachpb.Transaction{ + TxnMeta: txnMeta, + ReadTimestamp: ts, + } + intent := roachpb.MakeIntent(&txnMeta, key) + return txn, intent } func intents(intents ...roachpb.Intent) []roachpb.Intent { @@ -506,10 +504,10 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { testKey1 = roachpb.Key("/db1") testKey2 = roachpb.Key("/db2") - testValue1 = []byte("val1") - testValue2 = []byte("val2") - testValue3 = []byte("val3") - testValue4 = []byte("val4") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") + testValue3 = roachpb.MakeValueFromString("val3") + testValue4 = roachpb.MakeValueFromString("val4") // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. @@ -525,7 +523,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { kv1_2_2 := makeKVT(testKey1, testValue2, ts2) kv2_2_2 := makeKVT(testKey2, testValue3, ts2) kv2_4_4 := makeKVT(testKey2, testValue4, ts4) - kv1_3Deleted := makeKVT(testKey1, nil, ts3) + kv1_3Deleted := makeKVT(testKey1, roachpb.Value{}, ts3) for _, engineImpl := range mvccEngineImpls { t.Run(engineImpl.name, func(t *testing.T) { @@ -596,8 +594,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { }, ReadTimestamp: ts4, } - txn1Val := roachpb.Value{RawBytes: testValue4} - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { t.Fatal(err) } @@ -644,10 +641,10 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { testKey1 = roachpb.Key("/db1") testKey2 = roachpb.Key("/db2") - testValue1 = []byte("val1") - testValue2 = []byte("val2") - testValue3 = []byte("val3") - testValue4 = []byte("val4") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") + testValue3 = roachpb.MakeValueFromString("val3") + testValue4 = roachpb.MakeValueFromString("val4") // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. @@ -662,7 +659,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { kv1_1_1 := makeKVT(testKey1, testValue1, ts1) kv1_2_2 := makeKVT(testKey1, testValue2, ts2) kv2_2_2 := makeKVT(testKey2, testValue3, ts2) - kv1_3Deleted := makeKVT(testKey1, nil, ts3) + kv1_3Deleted := makeKVT(testKey1, roachpb.Value{}, ts3) for _, engineImpl := range mvccEngineImpls { t.Run(engineImpl.name, func(t *testing.T) { @@ -730,8 +727,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { }, ReadTimestamp: ts4, } - txn1Val := roachpb.Value{RawBytes: testValue4} - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { t.Fatal(err) } @@ -776,8 +772,8 @@ func TestMVCCIncrementalIteratorInlinePolicy(t *testing.T) { testKey2 = roachpb.Key("/db2") testKey3 = roachpb.Key("/db3") - testValue1 = []byte("val1") - testValue2 = []byte("val2") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. @@ -869,9 +865,9 @@ func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { testKey1 = roachpb.Key("/db1") testKey2 = roachpb.Key("/db2") - testValue1 = []byte("val1") - testValue2 = []byte("val2") - testValue3 = []byte("val3") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") + testValue3 = roachpb.MakeValueFromString("val3") // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. @@ -882,28 +878,12 @@ func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { tsMax = hlc.Timestamp{WallTime: math.MaxInt64, Logical: 0} ) - makeTxn := func(key roachpb.Key, val []byte, ts hlc.Timestamp) (roachpb.Transaction, roachpb.Value, roachpb.Intent) { - txnID := uuid.MakeV4() - txnMeta := enginepb.TxnMeta{ - Key: key, - ID: txnID, - Epoch: 1, - WriteTimestamp: ts, - } - return roachpb.Transaction{ - TxnMeta: txnMeta, - ReadTimestamp: ts, - }, roachpb.Value{ - RawBytes: val, - }, roachpb.MakeIntent(&txnMeta, key) - } - kv1_1_1 := makeKVT(testKey1, testValue1, ts1) kv1_2_2 := makeKVT(testKey1, testValue2, ts2) kv1_3_3 := makeKVT(testKey1, testValue3, ts3) kv2_1_1 := makeKVT(testKey2, testValue1, ts1) kv2_2_2 := makeKVT(testKey2, testValue2, ts2) - txn, val, intent2_2_2 := makeTxn(testKey2, testValue2, ts2) + txn, intent2_2_2 := makeKVTxn(testKey2, ts2) intentErr := &roachpb.WriteIntentError{Intents: []roachpb.Intent{intent2_2_2}} @@ -916,7 +896,7 @@ func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { t.Fatal(err) } } - if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val, &txn); err != nil { + if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, testValue2, &txn); err != nil { t.Fatal(err) } t.Run(engineImpl.name, func(t *testing.T) { @@ -1065,10 +1045,10 @@ func TestMVCCIncrementalIterator(t *testing.T) { testKey1 = roachpb.Key("/db1") testKey2 = roachpb.Key("/db2") - testValue1 = []byte("val1") - testValue2 = []byte("val2") - testValue3 = []byte("val3") - testValue4 = []byte("val4") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") + testValue3 = roachpb.MakeValueFromString("val3") + testValue4 = roachpb.MakeValueFromString("val4") // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. @@ -1085,7 +1065,7 @@ func TestMVCCIncrementalIterator(t *testing.T) { kv1_4_4 := makeKVT(testKey1, testValue4, ts4) kv1_2_2 := makeKVT(testKey1, testValue2, ts2) kv2_2_2 := makeKVT(testKey2, testValue3, ts2) - kv1Deleted3 := makeKVT(testKey1, nil, ts3) + kv1Deleted3 := makeKVT(testKey1, roachpb.Value{}, ts3) for _, engineImpl := range mvccEngineImpls { t.Run(engineImpl.name+"-latest", func(t *testing.T) { @@ -1120,12 +1100,12 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, latest, kvs(kv1Deleted3, kv2_2_2))) // Exercise intent handling. - txn1, txn1Val, intentErr1 := makeKVTxn(testKey1, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { + txn1, intentErr1 := makeKVTxn(testKey1, ts4) + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { t.Fatal(err) } - txn2, txn2Val, intentErr2 := makeKVTxn(testKey2, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, txn2Val, &txn2); err != nil { + txn2, intentErr2 := makeKVTxn(testKey2, ts4) + if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn2); err != nil { t.Fatal(err) } t.Run("intents-1", @@ -1188,12 +1168,12 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, all, kvs(kv1Deleted3, kv1_2_2, kv2_2_2))) // Exercise intent handling. - txn1, txn1Val, intentErr1 := makeKVTxn(testKey1, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { + txn1, intentErr1 := makeKVTxn(testKey1, ts4) + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { t.Fatal(err) } - txn2, txn2Val, intentErr2 := makeKVTxn(testKey2, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, txn2Val, &txn2); err != nil { + txn2, intentErr2 := makeKVTxn(testKey2, ts4) + if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn2); err != nil { t.Fatal(err) } // Single intent tests are verifying behavior when intent collection is not enabled. diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 40d75745af12..584fa9791632 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -36,7 +36,14 @@ const ( ) // MVCCKey is a versioned key, distinguished from roachpb.Key with the addition -// of a timestamp. +// of a "version" timestamp. +// +// The version timestamp dictates the key's visibility to readers. Readers with +// read timestamps equal to or greater than the version timestamp observe the +// key. Readers with read timestamps below the version timestamp ignore the key. +// Keys are stored in decreasing version order, with the exception of version +// zero (timestamp 0), which is referred to as a "meta" version and is stored +// before all other versions of the same key. type MVCCKey struct { Key roachpb.Key Timestamp hlc.Timestamp @@ -246,6 +253,7 @@ func encodedMVCCKeyLength(key MVCCKey) int { if key.Timestamp.Logical != 0 || key.Timestamp.Synthetic { keyLen += mvccEncodedTimeLogicalLen if key.Timestamp.Synthetic { + // TODO(nvanbenschoten): stop writing Synthetic timestamps in v23.1. keyLen += mvccEncodedTimeSyntheticLen } } @@ -307,6 +315,7 @@ func decodeMVCCTimestamp(encodedTS []byte) (hlc.Timestamp, error) { case 13: ts.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) ts.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + // TODO(nvanbenschoten): stop writing Synthetic timestamps in v23.1. ts.Synthetic = encodedTS[12] != 0 default: return hlc.Timestamp{}, errors.Errorf("bad timestamp %x", encodedTS) diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 855d5e9fe01d..2d45b747c525 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -35,6 +35,7 @@ import ( // implementations and errors on mismatch with any of them. It is used for global // keys. func assertEq(t *testing.T, rw ReadWriter, debug string, ms, expMS *enginepb.MVCCStats) { + t.Helper() assertEqImpl(t, rw, debug, true /* globalKeys */, ms, expMS) } @@ -137,6 +138,13 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { t.Fatal(err) } + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize += vValHeaderSize // 23 + expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 4e9, LiveBytes: 0, @@ -146,7 +154,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { // The implicit meta record (deletion tombstone) counts for len("a")+1=2. // Two versioned keys count for 2*vKeySize. KeyBytes: mKeySize + 2*vKeySize, - ValBytes: vValSize, // the initial write (10) + ValBytes: vValSize, // No GCBytesAge has been accrued yet, as the value just got non-live at 4s. GCBytesAge: 0, } @@ -185,7 +193,7 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { } mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - mValSize := int64((&enginepb.MVCCMetadata{ // 44 + mValSize := int64((&enginepb.MVCCMetadata{ // 46 Timestamp: ts1.ToLegacyTimestamp(), Deleted: false, Txn: &txn.TxnMeta, @@ -196,11 +204,11 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(44[+2])+12+10 = 68[+2] + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+10 = 68[+2] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 2+12 =14 KeyCount: 1, - ValBytes: mValSize + vValSize, // (44[+2])+10 = 54[+2] + ValBytes: mValSize + vValSize, // (46[+2])+10 = 54[+2] ValCount: 1, IntentCount: 1, SeparatedIntentCount: 1, @@ -220,9 +228,16 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { t.Fatal(err) } + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize += vValHeaderSize // 23 + expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 4e9, - LiveBytes: mKeySize + vKeySize + vValSize, // 2+12+20 = 24 + LiveBytes: mKeySize + vKeySize + vValSize, // 2+12+23 = 37 LiveCount: 1, KeyCount: 1, ValCount: 1, @@ -267,7 +282,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { } mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - mValSize := int64((&enginepb.MVCCMetadata{ // 44 + mValSize := int64((&enginepb.MVCCMetadata{ // 46 Timestamp: ts1.ToLegacyTimestamp(), Deleted: false, Txn: &txn.TxnMeta, @@ -278,11 +293,11 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(44[+2])+12+10 = 68[+2] + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+10 = 70[+2] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 2+12 = 14 KeyCount: 1, - ValBytes: mValSize + vValSize, // (44[+2])+10 = 54[+2] + ValBytes: mValSize + vValSize, // (46[+2])+10 = 54[+2] ValCount: 1, IntentAge: 0, IntentCount: 1, @@ -300,12 +315,19 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { ); err != nil { t.Fatal(err) } - // Account for removal of TxnDidNotUpdateMeta + // Account for removal of TxnDidNotUpdateMeta. mValSize -= 2 + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize += vValHeaderSize // 23 + expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 4e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+44+12+20 = 78 + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+54+12+23 = 91 LiveCount: 1, KeyCount: 1, ValCount: 1, @@ -313,7 +335,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { // One versioned key counts for vKeySize. KeyBytes: mKeySize + vKeySize, // The intent is still there, so we see mValSize. - ValBytes: vValSize + mValSize, // 44+10 = 54 + ValBytes: mValSize + vValSize, // 54+23 = 69 IntentAge: 0, // this was once erroneously positive IntentCount: 1, // still there SeparatedIntentCount: 1, @@ -647,11 +669,18 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { t.Fatal(err) } + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts2)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize := vValHeaderSize + 0 // tombstone, so just a header + expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 3e9, KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 KeyCount: 1, - ValBytes: 0, + ValBytes: vValSize, ValCount: 2, IntentCount: 0, IntentBytes: 0, @@ -980,7 +1009,7 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { } mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - m1ValSize := int64((&enginepb.MVCCMetadata{ // 44 + m1ValSize := int64((&enginepb.MVCCMetadata{ // 46 Timestamp: ts201.ToLegacyTimestamp(), Txn: &txn.TxnMeta, }).Size()) @@ -990,11 +1019,11 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { expMS := enginepb.MVCCStats{ LastUpdateNanos: 2e9 + 1, - LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(44[+2])+12+10 = 68[+2] + LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(46[+2])+12+10 = 68[+2] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 14 KeyCount: 1, - ValBytes: m1ValSize + vValSize, // (44[+2])+10 = 54[+2] + ValBytes: m1ValSize + vValSize, // (46[+2])+10 = 54[+2] ValCount: 1, IntentCount: 1, SeparatedIntentCount: 1, diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 6060927203a2..8598b3c9c656 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -960,23 +960,24 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := engine.PutMVCC(MVCCKey{testKey1, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + value := MVCCValue{Value: roachpb.MakeValueFromString("foobar")} + err := engine.PutMVCC(MVCCKey{Key: testKey1, Timestamp: hlc.Timestamp{WallTime: 5}}, value) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey2, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{Key: testKey2, Timestamp: hlc.Timestamp{WallTime: 5}}, value) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey2, hlc.Timestamp{WallTime: 3}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{Key: testKey2, Timestamp: hlc.Timestamp{WallTime: 3}}, value) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey3, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{Key: testKey3, Timestamp: hlc.Timestamp{WallTime: 5}}, value) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey4, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{Key: testKey4, Timestamp: hlc.Timestamp{WallTime: 5}}, value) if err != nil { t.Fatal(err) } @@ -997,7 +998,7 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { UpperBound: testKey5, }) defer iter.Close() - iter.SeekGE(MVCCKey{testKey1, hlc.Timestamp{WallTime: 5}}) + iter.SeekGE(MVCCKey{Key: testKey1, Timestamp: hlc.Timestamp{WallTime: 5}}) iter.Next() // key2/5 // Lay down an intent on key3, which will go at key3/0 and sort before key3/5. @@ -4137,8 +4138,8 @@ func TestRandomizedMVCCResolveWriteIntentRange(t *testing.T) { // non-decreasing due to tsIndex. versions := rng.Intn(3) + 1 for j := 0; j < versions; j++ { - val := generateBytes(rng, 20, 30) - put.values = append(put.values, roachpb.Value{RawBytes: val}) + val := roachpb.MakeValueFromBytes(generateBytes(rng, 20, 30)) + put.values = append(put.values, val) put.seqs = append(put.seqs, seq) seq++ index := rng.Intn(len(timestamps)) @@ -4261,8 +4262,8 @@ func TestRandomizedSavepointRollbackAndIntentResolution(t *testing.T) { key: key, } for j := 0; j < 2; j++ { - val := generateBytes(rng, 20, 30) - put.values = append(put.values, roachpb.Value{RawBytes: val}) + val := roachpb.MakeValueFromBytes(generateBytes(rng, 20, 30)) + put.values = append(put.values, val) put.seqs = append(put.seqs, seq) seq++ put.writeTS = append(put.writeTS, timestamps[j]) @@ -4310,8 +4311,8 @@ func TestRandomizedSavepointRollbackAndIntentResolution(t *testing.T) { } // Do another put for all these keys. These will also be in the memtable. for i := 0; i < 100; i++ { - puts[i].values = append(puts[i].values[:0], - roachpb.Value{RawBytes: generateBytes(rng, 2, 3)}) + val := roachpb.MakeValueFromBytes(generateBytes(rng, 2, 3)) + puts[i].values = append(puts[i].values[:0], val) puts[i].seqs = append(puts[i].seqs[:0], seq) seq++ puts[i].writeTS = append(puts[i].writeTS[:0], timestamps[2]) diff --git a/pkg/storage/mvcc_value.go b/pkg/storage/mvcc_value.go new file mode 100644 index 000000000000..86ea768eb0b6 --- /dev/null +++ b/pkg/storage/mvcc_value.go @@ -0,0 +1,245 @@ +// 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 storage + +import ( + "encoding/binary" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" +) + +const ( + extendedLenSize = 4 // also checksumSize for roachpb.Value + tagPos = extendedLenSize + tagSize = 1 + extendedPreludeSize = extendedLenSize + tagSize + + extendedEncodingSentinel = byte(roachpb.ValueType_MVCC_EXTENDED_ENCODING_SENTINEL) +) + +// MVCCValue is a versioned value, stored at an associated MVCCKey with a +// non-zero version timestamp. +// +// MVCCValue wraps a roachpb.Value and extends it with MVCC-level metadata which +// is stored in an enginepb.MVCCValueHeader struct. +// +// The mvcc value has a "simple" and an "extended" encoding scheme, depending on +// whether the value's header is empty or not. If the value's header is empty, +// it is omitted in the encoding and the mvcc value's encoding is identical to +// that of roachpb.Value. This provided backwards compatibility and ensures that +// the MVCCValue optimizes away in the common case. If the value's header is not +// empty, it is prepended to the roachpb.Value encoding. The encoding scheme's +// variants are: +// +// Simple (identical to the roachpb.Value encoding): +// +// <4-byte-checksum><1-byte-tag> +// +// Extended (header prepended to roachpb.Value encoding): +// +// <4-byte-header-len><1-byte-sentinel><4-byte-checksum><1-byte-tag> +// +// The two encoding scheme variants are distinguished using the 5th byte, which +// is either the roachpb.Value tag (which has many values) or a sentinel tag not +// used by the roachpb.Value encoding which indicates the extended encoding +// scheme. +// +// For a deletion tombstone, the encoding of roachpb.Value is special cased to +// be empty, i.e., no checksum, tag, or encoded-data. In that case the extended +// encoding above is simply: +// +// <4-byte-header-len><1-byte-sentinel> +// +// To identify a deletion tombstone from an encoded MVCCValue, callers should +// decode the value using DecodeMVCCValue and then use the IsTombstone method. +// For example: +// +// valRaw := iter.UnsafeValue() +// val, err := DecodeMVCCValue(valRaw) +// if err != nil { ... } +// isTombstone := val.IsTombstone() +// +type MVCCValue struct { + enginepb.MVCCValueHeader + Value roachpb.Value +} + +// IsTombstone returns whether the MVCCValue represents a deletion tombstone. +func (v MVCCValue) IsTombstone() bool { + return len(v.Value.RawBytes) == 0 +} + +// LocalTimestampNeeded returns whether the MVCCValue's local timestamp is +// needed, or whether it can be implied by (i.e. set to the same value as) +// its key's version timestamp. +func (v MVCCValue) LocalTimestampNeeded(k MVCCKey) bool { + // If the local timestamp is empty, it is assumed to be equal to the key's + // version timestamp and so the local timestamp is not needed. + return !v.LocalTimestamp.IsEmpty() && + // If the local timestamp is not empty, it is safe for the local clock + // timestamp to be rounded down, as this will simply lead to additional + // uncertainty restarts. In such cases, the local timestamp is not needed. + // However, it is not safe for the local clock timestamp to be rounded up, + // as this could lead to stale reads. As a result, in such cases, the local + // timestamp is needed and cannot be implied by the version timestamp. + v.LocalTimestamp.ToTimestamp().Less(k.Timestamp) +} + +// GetLocalTimestamp returns the MVCCValue's local timestamp. If the local +// timestamp is not set explicitly, its implicit value is taken from the +// provided MVCCKey and returned. +func (v MVCCValue) GetLocalTimestamp(k MVCCKey) hlc.ClockTimestamp { + if v.LocalTimestamp.IsEmpty() { + if k.Timestamp.Synthetic { + // A synthetic version timestamp means that the version timestamp is + // disconnected from real time and did not come from an HLC clock on the + // leaseholder that wrote the value or from somewhere else in the system. + // As a result, the version timestamp cannot be cast to a clock timestamp, + // so we return min_clock_timestamp instead. The effect of this is that + // observed timestamps can not be used to avoid uncertainty retries for + // values without a local timestamp and with a synthetic version + // timestamp. + return hlc.MinClockTimestamp + } + return hlc.ClockTimestamp(k.Timestamp) + } + return v.LocalTimestamp +} + +// String implements the fmt.Stringer interface. +func (v MVCCValue) String() string { + return redact.StringWithoutMarkers(v) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (v MVCCValue) SafeFormat(w redact.SafePrinter, _ rune) { + if v.MVCCValueHeader != (enginepb.MVCCValueHeader{}) { + w.Printf("vheader{") + if !v.LocalTimestamp.IsEmpty() { + w.Printf(" localTs=%s", v.LocalTimestamp) + } + w.Printf(" } ") + } + w.Print(v.Value.PrettyPrint()) +} + +// encodedMVCCValueSize returns the size of the MVCCValue when encoded. +//gcassert:inline +func encodedMVCCValueSize(v MVCCValue) int { + if v.MVCCValueHeader == (enginepb.MVCCValueHeader{}) { + return len(v.Value.RawBytes) + } + return extendedPreludeSize + v.MVCCValueHeader.Size() + len(v.Value.RawBytes) +} + +// EncodeMVCCValue encodes an MVCCValue into its Pebble representation. See the +// comment on MVCCValue for a description of the encoding scheme. +//gcassert:inline +func EncodeMVCCValue(v MVCCValue) ([]byte, error) { + if v.MVCCValueHeader == (enginepb.MVCCValueHeader{}) { + // Simple encoding. Use the roachpb.Value encoding directly with no + // modification. No need to re-allocate or copy. + return v.Value.RawBytes, nil + } + // Extended encoding. Wrap the roachpb.Value encoding with a header containing + // MVCC-level metadata. Requires a re-allocation and copy. + return encodeExtendedMVCCValue(v) +} + +// encodeExtendedMVCCValue implements the extended MVCCValue encoding. It is +// split from EncodeMVCCValue to allow that function to qualify for mid-stack +// inlining, which avoids a function call for the simple encoding. +func encodeExtendedMVCCValue(v MVCCValue) ([]byte, error) { + headerLen := v.MVCCValueHeader.Size() + headerSize := extendedPreludeSize + headerLen + valueSize := headerSize + len(v.Value.RawBytes) + + buf := make([]byte, valueSize) + // 4-byte-header-len + binary.BigEndian.PutUint32(buf, uint32(headerLen)) + // 1-byte-sentinel + buf[tagPos] = extendedEncodingSentinel + // mvcc-header + // + // NOTE: we don't use protoutil to avoid passing v.MVCCValueHeader through + // an interface, which would cause a heap allocation and incur the cost of + // dynamic dispatch. + if _, err := v.MVCCValueHeader.MarshalToSizedBuffer(buf[extendedPreludeSize:headerSize]); err != nil { + return nil, errors.Wrap(err, "marshaling MVCCValueHeader") + } + // <4-byte-checksum><1-byte-tag> or empty for tombstone + copy(buf[headerSize:], v.Value.RawBytes) + return buf, nil +} + +// DecodeMVCCValue decodes an MVCCKey from its Pebble representation. +// +// NOTE: this function does not inline, so it is not suitable for performance +// critical code paths. Instead, callers that care about performance and would +// like to avoid function calls should manually call the two decoding functions. +// tryDecodeSimpleMVCCValue does inline, so callers can use it to avoid making +// any function calls when decoding an MVCCValue that is encoded with the simple +// encoding. +func DecodeMVCCValue(buf []byte) (MVCCValue, error) { + v, ok, err := tryDecodeSimpleMVCCValue(buf) + if ok || err != nil { + return v, err + } + return decodeExtendedMVCCValue(buf) +} + +// Static error definitions, to permit inlining. +var errMVCCValueMissingTag = errors.Errorf("invalid encoded mvcc value, missing tag") +var errMVCCValueMissingHeader = errors.Errorf("invalid encoded mvcc value, missing header") + +// tryDecodeSimpleMVCCValue attempts to decode an MVCCValue that is using the +// simple encoding. If successful, returns the decoded value and true. If the +// value was using the extended encoding, returns false, in which case the +// caller should call decodeExtendedMVCCValue. +//gcassert:inline +func tryDecodeSimpleMVCCValue(buf []byte) (MVCCValue, bool, error) { + if len(buf) == 0 { + // Tombstone with no header. + return MVCCValue{}, true, nil + } + if len(buf) <= tagPos { + return MVCCValue{}, false, errMVCCValueMissingTag + } + if buf[tagPos] != extendedEncodingSentinel { + // Simple encoding. The encoding is equivalent to the roachpb.Value + // encoding, so inflate it directly. No need to copy or slice. + return MVCCValue{Value: roachpb.Value{RawBytes: buf}}, true, nil + } + // Extended encoding. The caller should call decodeExtendedMVCCValue. + return MVCCValue{}, false, nil +} + +func decodeExtendedMVCCValue(buf []byte) (MVCCValue, error) { + headerLen := binary.BigEndian.Uint32(buf) + headerSize := extendedPreludeSize + headerLen + if len(buf) < int(headerSize) { + return MVCCValue{}, errMVCCValueMissingHeader + } + var header enginepb.MVCCValueHeader + // NOTE: we don't use protoutil to avoid passing header through an interface, + // which would cause a heap allocation and incur the cost of dynamic dispatch. + if err := header.Unmarshal(buf[extendedPreludeSize:headerSize]); err != nil { + return MVCCValue{}, errors.Wrapf(err, "unmarshaling MVCCValueHeader") + } + var v MVCCValue + v.LocalTimestamp = header.LocalTimestamp + v.Value.RawBytes = buf[headerSize:] + return v, nil +} diff --git a/pkg/storage/mvcc_value_test.go b/pkg/storage/mvcc_value_test.go new file mode 100644 index 000000000000..fc8ee3473009 --- /dev/null +++ b/pkg/storage/mvcc_value_test.go @@ -0,0 +1,236 @@ +// 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 storage + +import ( + "bytes" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestMVCCValueLocalTimestampNeeded(t *testing.T) { + defer leaktest.AfterTest(t)() + + ts0 := hlc.Timestamp{Logical: 0} + ts1 := hlc.Timestamp{Logical: 1} + ts2 := hlc.Timestamp{Logical: 2} + + testcases := map[string]struct { + localTs hlc.Timestamp + versionTs hlc.Timestamp + expect bool + }{ + "no local timestamp": {ts0, ts2, false}, + "smaller local timestamp": {ts1, ts2, true}, + "equal local timestamp": {ts2, ts2, false}, + "larger local timestamp": {ts2, ts1, false}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + mvccKey := MVCCKey{Timestamp: tc.versionTs} + mvccVal := MVCCValue{} + mvccVal.LocalTimestamp = hlc.ClockTimestamp(tc.localTs) + + require.Equal(t, tc.expect, mvccVal.LocalTimestampNeeded(mvccKey)) + }) + } +} + +func TestMVCCValueGetLocalTimestamp(t *testing.T) { + defer leaktest.AfterTest(t)() + + ts0 := hlc.Timestamp{Logical: 0} + ts1 := hlc.Timestamp{Logical: 1} + ts2 := hlc.Timestamp{Logical: 2} + ts2S := hlc.Timestamp{Logical: 2, Synthetic: true} + + testcases := map[string]struct { + localTs hlc.Timestamp + versionTs hlc.Timestamp + expect hlc.Timestamp + }{ + "no local timestamp": {ts0, ts2, ts2}, + "no local timestamp, version synthetic": {ts0, ts2S, hlc.MinTimestamp}, + "smaller local timestamp": {ts1, ts2, ts1}, + "equal local timestamp": {ts2, ts2, ts2}, + "larger local timestamp": {ts2, ts1, ts2}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + mvccKey := MVCCKey{Timestamp: tc.versionTs} + mvccVal := MVCCValue{} + mvccVal.LocalTimestamp = hlc.ClockTimestamp(tc.localTs) + + require.Equal(t, hlc.ClockTimestamp(tc.expect), mvccVal.GetLocalTimestamp(mvccKey)) + }) + } +} + +func TestMVCCValueFormat(t *testing.T) { + defer leaktest.AfterTest(t)() + + var strVal, intVal roachpb.Value + strVal.SetString("foo") + intVal.SetInt(17) + + valHeader := enginepb.MVCCValueHeader{} + valHeader.LocalTimestamp = hlc.ClockTimestamp{WallTime: 9} + + testcases := map[string]struct { + val MVCCValue + expect string + }{ + "tombstone": {val: MVCCValue{}, expect: "/"}, + "bytes": {val: MVCCValue{Value: strVal}, expect: "/BYTES/foo"}, + "int": {val: MVCCValue{Value: intVal}, expect: "/INT/17"}, + "header+tombstone": {val: MVCCValue{MVCCValueHeader: valHeader}, expect: "vheader{ localTs=0.000000009,0 } /"}, + "header+bytes": {val: MVCCValue{MVCCValueHeader: valHeader, Value: strVal}, expect: "vheader{ localTs=0.000000009,0 } /BYTES/foo"}, + "header+int": {val: MVCCValue{MVCCValueHeader: valHeader, Value: intVal}, expect: "vheader{ localTs=0.000000009,0 } /INT/17"}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + require.Equal(t, tc.expect, tc.val.String()) + }) + } +} + +func TestEncodeDecodeMVCCValue(t *testing.T) { + defer leaktest.AfterTest(t)() + + var strVal, intVal roachpb.Value + strVal.SetString("foo") + intVal.SetInt(17) + + valHeader := enginepb.MVCCValueHeader{} + valHeader.LocalTimestamp = hlc.ClockTimestamp{WallTime: 9} + + testcases := map[string]struct { + val MVCCValue + expect []byte + }{ + "tombstone": {val: MVCCValue{}, expect: nil}, + "bytes": {val: MVCCValue{Value: strVal}, expect: []byte{0x0, 0x0, 0x0, 0x0, 0x3, 0x66, 0x6f, 0x6f}}, + "int": {val: MVCCValue{Value: intVal}, expect: []byte{0x0, 0x0, 0x0, 0x0, 0x1, 0x22}}, + "header+tombstone": {val: MVCCValue{MVCCValueHeader: valHeader}, expect: []byte{0x0, 0x0, 0x0, 0x4, 0x65, 0xa, 0x2, 0x8, 0x9}}, + "header+bytes": {val: MVCCValue{MVCCValueHeader: valHeader, Value: strVal}, expect: []byte{0x0, 0x0, 0x0, 0x4, 0x65, 0xa, 0x2, 0x8, 0x9, 0x0, 0x0, 0x0, 0x0, 0x3, 0x66, 0x6f, 0x6f}}, + "header+int": {val: MVCCValue{MVCCValueHeader: valHeader, Value: intVal}, expect: []byte{0x0, 0x0, 0x0, 0x4, 0x65, 0xa, 0x2, 0x8, 0x9, 0x0, 0x0, 0x0, 0x0, 0x1, 0x22}}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + encSize := encodedMVCCValueSize(tc.val) + require.Equal(t, len(tc.expect), encSize) + + enc, err := EncodeMVCCValue(tc.val) + require.NoError(t, err) + require.Equal(t, tc.expect, enc) + + dec, err := DecodeMVCCValue(enc) + require.NoError(t, err) + if len(dec.Value.RawBytes) == 0 { + dec.Value.RawBytes = nil // normalize + } + require.Equal(t, tc.val, dec) + }) + } +} + +func TestDecodeMVCCValueErrors(t *testing.T) { + defer leaktest.AfterTest(t)() + + testcases := map[string]struct { + enc []byte + expect error + }{ + "missing tag": {enc: []byte{0x0}, expect: errMVCCValueMissingTag}, + "missing header": {enc: []byte{0x0, 0x0, 0x0, 0x1, extendedEncodingSentinel}, expect: errMVCCValueMissingHeader}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + dec, err := DecodeMVCCValue(tc.enc) + require.Equal(t, tc.expect, err) + require.Zero(t, dec) + }) + } +} + +var mvccValueBenchmarkConfigs = struct { + headers map[string]enginepb.MVCCValueHeader + values map[string]roachpb.Value +}{ + headers: map[string]enginepb.MVCCValueHeader{ + "empty": {}, + "local walltime": {LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652545}}, + "local walltime+logical": {LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 4096}}, + }, + values: map[string]roachpb.Value{ + "tombstone": {}, + "short": roachpb.MakeValueFromString("foo"), + "long": roachpb.MakeValueFromBytes(bytes.Repeat([]byte{1}, 4096)), + }, +} + +func BenchmarkEncodeMVCCValue(b *testing.B) { + cfg := mvccValueBenchmarkConfigs + for hDesc, h := range cfg.headers { + for vDesc, v := range cfg.values { + name := fmt.Sprintf("header=%s/value=%s", hDesc, vDesc) + mvccValue := MVCCValue{MVCCValueHeader: h, Value: v} + b.Run(name, func(b *testing.B) { + for i := 0; i < b.N; i++ { + res, err := EncodeMVCCValue(mvccValue) + if err != nil { // for performance + require.NoError(b, err) + } + _ = res + } + }) + } + } +} + +func BenchmarkDecodeMVCCValue(b *testing.B) { + cfg := mvccValueBenchmarkConfigs + for hDesc, h := range cfg.headers { + for vDesc, v := range cfg.values { + for _, inline := range []bool{false, true} { + name := fmt.Sprintf("header=%s/value=%s/inline=%t", hDesc, vDesc, inline) + mvccValue := MVCCValue{MVCCValueHeader: h, Value: v} + buf, err := EncodeMVCCValue(mvccValue) + require.NoError(b, err) + b.Run(name, func(b *testing.B) { + for i := 0; i < b.N; i++ { + var res MVCCValue + var err error + if inline { + var ok bool + res, ok, err = tryDecodeSimpleMVCCValue(buf) + if !ok && err == nil { + res, err = decodeExtendedMVCCValue(buf) + } + } else { + res, err = DecodeMVCCValue(buf) + } + if err != nil { // for performance + require.NoError(b, err) + } + _ = res + } + }) + } + } + } +} diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index d9dc4d265020..6dbc0fec1e64 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1246,10 +1246,22 @@ func (p *Pebble) Merge(key MVCCKey, value []byte) error { } // PutMVCC implements the Engine interface. -func (p *Pebble) PutMVCC(key MVCCKey, value []byte) error { +func (p *Pebble) PutMVCC(key MVCCKey, value MVCCValue) error { if key.Timestamp.IsEmpty() { panic("PutMVCC timestamp is empty") } + encValue, err := EncodeMVCCValue(value) + if err != nil { + return err + } + return p.put(key, encValue) +} + +// PutRawMVCC implements the Engine interface. +func (p *Pebble) PutRawMVCC(key MVCCKey, value []byte) error { + if key.Timestamp.IsEmpty() { + panic("PutRawMVCC timestamp is empty") + } return p.put(key, value) } @@ -2071,7 +2083,11 @@ func (p *pebbleReadOnly) Merge(key MVCCKey, value []byte) error { panic("not implemented") } -func (p *pebbleReadOnly) PutMVCC(key MVCCKey, value []byte) error { +func (p *pebbleReadOnly) PutMVCC(key MVCCKey, value MVCCValue) error { + panic("not implemented") +} + +func (p *pebbleReadOnly) PutRawMVCC(key MVCCKey, value []byte) error { panic("not implemented") } @@ -2286,7 +2302,6 @@ func pebbleExportToSst( break } - unsafeValue := iter.UnsafeValue() isNewKey := !options.ExportAllRevisions || !unsafeKey.Key.Equal(curKey) if trackKeyBoundary && options.ExportAllRevisions && isNewKey { curKey = append(curKey[:0], unsafeKey.Key...) @@ -2319,10 +2334,27 @@ func pebbleExportToSst( } } - // Skip tombstone (len=0) records when start time is zero (non-incremental) - // and we are not exporting all versions. - skipTombstones := !options.ExportAllRevisions && options.StartTS.IsEmpty() - if len(unsafeValue) > 0 || !skipTombstones { + unsafeValue := iter.UnsafeValue() + skip := false + if unsafeKey.IsValue() { + mvccValue, ok, err := tryDecodeSimpleMVCCValue(unsafeValue) + if !ok && err == nil { + mvccValue, err = decodeExtendedMVCCValue(unsafeValue) + } + if err != nil { + return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding mvcc value %s", unsafeKey) + } + + // Export only the inner roachpb.Value, not the MVCCValue header. + unsafeValue = mvccValue.Value.RawBytes + + // Skip tombstone records when start time is zero (non-incremental) + // and we are not exporting all versions. + skipTombstones := !options.ExportAllRevisions && options.StartTS.IsEmpty() + skip = skipTombstones && mvccValue.IsTombstone() + } + + if !skip { if err := rows.Count(unsafeKey.Key); err != nil { return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding %s", unsafeKey) } @@ -2351,7 +2383,7 @@ func pebbleExportToSst( return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "adding key %s", unsafeKey) } } else { - if err := sstWriter.PutMVCC(unsafeKey, unsafeValue); err != nil { + if err := sstWriter.PutRawMVCC(unsafeKey, unsafeValue); err != nil { return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "adding key %s", unsafeKey) } } diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index ffb853472074..2ac471bc2188 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -438,10 +438,22 @@ func (p *pebbleBatch) Merge(key MVCCKey, value []byte) error { } // PutMVCC implements the Batch interface. -func (p *pebbleBatch) PutMVCC(key MVCCKey, value []byte) error { +func (p *pebbleBatch) PutMVCC(key MVCCKey, value MVCCValue) error { if key.Timestamp.IsEmpty() { panic("PutMVCC timestamp is empty") } + encValue, err := EncodeMVCCValue(value) + if err != nil { + return err + } + return p.put(key, encValue) +} + +// PutRawMVCC implements the Batch interface. +func (p *pebbleBatch) PutRawMVCC(key MVCCKey, value []byte) error { + if key.Timestamp.IsEmpty() { + panic("PutRawMVCC timestamp is empty") + } return p.put(key, value) } diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index f1897da83e14..94c2d9e1a959 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -345,11 +345,12 @@ type pebbleMVCCScanner struct { // cur* variables store the "current" record we're pointing to. Updated in // updateCurrent. Note that the timestamp can be clobbered in the case of // adding an intent from the intent history but is otherwise meaningful. - curUnsafeKey MVCCKey - curRawKey []byte - curValue []byte - results pebbleResults - intents pebble.Batch + curUnsafeKey MVCCKey + curRawKey []byte + curUnsafeValue MVCCValue + curRawValue []byte + results pebbleResults + intents pebble.Batch // mostRecentTS stores the largest timestamp observed that is equal to or // above the scan timestamp. Only applicable if failOnMoreRecent is true. If // set and no other error is hit, a WriteToOld error will be returned from @@ -548,11 +549,19 @@ func (p *pebbleMVCCScanner) uncertaintyError(ts hlc.Timestamp) bool { // continue. func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { if !p.curUnsafeKey.Timestamp.IsEmpty() { + if extended, valid := p.tryDecodeCurrentValueSimple(); !valid { + return false + } else if extended { + if !p.decodeCurrentValueExtended() { + return false + } + } + // ts < read_ts if p.curUnsafeKey.Timestamp.Less(p.ts) { // 1. Fast path: there is no intent and our read timestamp is newer // than the most recent version's timestamp. - return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curUnsafeValue.Value.RawBytes) } // ts == read_ts @@ -573,7 +582,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // 3. There is no intent and our read timestamp is equal to the most // recent version's timestamp. - return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curUnsafeValue.Value.RawBytes) } // ts > read_ts @@ -595,7 +604,8 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // 5. Our txn's read timestamp is less than the max timestamp // seen by the txn. We need to check for clock uncertainty // errors. - if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp) { + localTS := p.curUnsafeValue.GetLocalTimestamp(p.curUnsafeKey) + if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp, localTS) { return p.uncertaintyError(p.curUnsafeKey.Timestamp) } @@ -613,13 +623,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { return p.seekVersion(ctx, p.ts, false) } - if len(p.curValue) == 0 { - p.err = errors.Errorf("zero-length mvcc metadata") - return false - } - err := protoutil.Unmarshal(p.curValue, &p.meta) - if err != nil { - p.err = errors.Wrap(err, "unable to decode MVCCMetadata") + if !p.decodeCurrentMetadata() { return false } if len(p.meta.RawBytes) != 0 { @@ -675,11 +679,11 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // p.intents is a pebble.Batch which grows its byte slice capacity in // chunks to amortize allocations. The memMonitor is under-counting here // by only accounting for the key and value bytes. - if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curValue))); p.err != nil { + if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curRawValue))); p.err != nil { p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) return false } - p.err = p.intents.Set(p.curRawKey, p.curValue, nil) + p.err = p.intents.Set(p.curRawKey, p.curRawValue, nil) if p.err != nil { return false } @@ -703,11 +707,11 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // p.intents is a pebble.Batch which grows its byte slice capacity in // chunks to amortize allocations. The memMonitor is under-counting here // by only accounting for the key and value bytes. - if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curValue))); p.err != nil { + if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curRawValue))); p.err != nil { p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) return false } - p.err = p.intents.Set(p.curRawKey, p.curValue, nil) + p.err = p.intents.Set(p.curRawKey, p.curRawValue, nil) if p.err != nil { return false } @@ -899,19 +903,19 @@ func (p *pebbleMVCCScanner) advanceKeyAtNewKey(key []byte) bool { // p.tombstones is true. Advances to the next key unless we've reached the max // results limit. func (p *pebbleMVCCScanner) addAndAdvance( - ctx context.Context, key roachpb.Key, rawKey []byte, val []byte, + ctx context.Context, key roachpb.Key, rawKey []byte, rawValue []byte, ) bool { // Don't include deleted versions len(val) == 0, unless we've been instructed // to include tombstones in the results. - if len(val) == 0 && !p.tombstones { + if len(rawValue) == 0 && !p.tombstones { return p.advanceKey() } // Check if adding the key would exceed a limit. if p.targetBytes > 0 && (p.results.bytes >= p.targetBytes || (p.targetBytesAvoidExcess && - p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(val))) > p.targetBytes)) { + p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(rawValue))) > p.targetBytes)) { p.resumeReason = roachpb.RESUME_BYTE_LIMIT - p.resumeNextBytes = int64(p.results.sizeOf(len(rawKey), len(val))) + p.resumeNextBytes = int64(p.results.sizeOf(len(rawKey), len(rawValue))) } else if p.maxKeys > 0 && p.results.count >= p.maxKeys { p.resumeReason = roachpb.RESUME_KEY_LIMIT @@ -943,7 +947,7 @@ func (p *pebbleMVCCScanner) addAndAdvance( } } - if err := p.results.put(ctx, rawKey, val, p.memAccount); err != nil { + if err := p.results.put(ctx, rawKey, rawValue, p.memAccount); err != nil { p.err = errors.Wrapf(err, "scan with start key %s", p.start) return false } @@ -999,8 +1003,15 @@ func (p *pebbleMVCCScanner) seekVersion( } if p.curUnsafeKey.Timestamp.LessEq(seekTS) { p.incrementItersBeforeSeek() + if extended, valid := p.tryDecodeCurrentValueSimple(); !valid { + return false + } else if extended { + if !p.decodeCurrentValueExtended() { + return false + } + } if !uncertaintyCheck || p.curUnsafeKey.Timestamp.LessEq(p.ts) { - return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curUnsafeValue.Value.RawBytes) } // Iterate through uncertainty interval. Though we found a value in // the interval, it may not be uncertainty. This is because seekTS @@ -1010,7 +1021,8 @@ func (p *pebbleMVCCScanner) seekVersion( // are only uncertain if their timestamps are synthetic. Meanwhile, // any value with a time in the range (ts, uncertainty.LocalLimit] // is uncertain. - if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp) { + localTS := p.curUnsafeValue.GetLocalTimestamp(p.curUnsafeKey) + if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp, localTS) { return p.uncertaintyError(p.curUnsafeKey.Timestamp) } } @@ -1024,13 +1036,21 @@ func (p *pebbleMVCCScanner) seekVersion( if !bytes.Equal(p.curUnsafeKey.Key, origKey) { return p.advanceKeyAtNewKey(origKey) } + if extended, valid := p.tryDecodeCurrentValueSimple(); !valid { + return false + } else if extended { + if !p.decodeCurrentValueExtended() { + return false + } + } if !uncertaintyCheck || p.curUnsafeKey.Timestamp.LessEq(p.ts) { - return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curUnsafeValue.Value.RawBytes) } // Iterate through uncertainty interval. See the comment above about why // a value in this interval is not necessarily cause for an uncertainty // error. - if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp) { + localTS := p.curUnsafeValue.GetLocalTimestamp(p.curUnsafeKey) + if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp, localTS) { return p.uncertaintyError(p.curUnsafeKey.Timestamp) } if !p.iterNext() { @@ -1039,7 +1059,9 @@ func (p *pebbleMVCCScanner) seekVersion( } } -// Updates cur{RawKey, Key, TS} to match record the iterator is pointing to. +// Updates cur{RawKey, UnsafeKey, RawValue} to match record the iterator is +// pointing to. Callers should call decodeCurrent{Metadata, Value} to decode +// the raw value if they need it. func (p *pebbleMVCCScanner) updateCurrent() bool { if !p.iterValid() { return false @@ -1050,12 +1072,45 @@ func (p *pebbleMVCCScanner) updateCurrent() bool { var err error p.curUnsafeKey, err = DecodeMVCCKey(p.curRawKey) if err != nil { - panic(err) + p.err = errors.Wrap(err, "unable to decode MVCCKey") + return false + } + p.curRawValue = p.parent.UnsafeValue() + + // Reset decoded value to avoid bugs. + if util.RaceEnabled { + p.meta = enginepb.MVCCMetadata{} + p.curUnsafeValue = MVCCValue{} } - p.curValue = p.parent.UnsafeValue() return true } +func (p *pebbleMVCCScanner) decodeCurrentMetadata() bool { + if len(p.curRawValue) == 0 { + p.err = errors.Errorf("zero-length mvcc metadata") + return false + } + err := protoutil.Unmarshal(p.curRawValue, &p.meta) + if err != nil { + p.err = errors.Wrap(err, "unable to decode MVCCMetadata") + return false + } + return true +} + +//gcassert:inline +func (p *pebbleMVCCScanner) tryDecodeCurrentValueSimple() (extended, valid bool) { + var simple bool + p.curUnsafeValue, simple, p.err = tryDecodeSimpleMVCCValue(p.curRawValue) + return !simple, p.err == nil +} + +//gcassert:inline +func (p *pebbleMVCCScanner) decodeCurrentValueExtended() bool { + p.curUnsafeValue, p.err = decodeExtendedMVCCValue(p.curRawValue) + return p.err == nil +} + func (p *pebbleMVCCScanner) iterValid() bool { if valid, err := p.parent.Valid(); !valid { // Defensive: unclear if p.err can already be non-nil here, but @@ -1138,9 +1193,9 @@ func (p *pebbleMVCCScanner) iterPeekPrev() ([]byte, bool) { // curRawKey, curKey and curValue to point to this saved data. We use a // single buffer for this purpose: savedBuf. p.savedBuf = append(p.savedBuf[:0], p.curRawKey...) - p.savedBuf = append(p.savedBuf, p.curValue...) + p.savedBuf = append(p.savedBuf, p.curRawValue...) p.curRawKey = p.savedBuf[:len(p.curRawKey)] - p.curValue = p.savedBuf[len(p.curRawKey):] + p.curRawValue = p.savedBuf[len(p.curRawKey):] // The raw key is always a prefix of the encoded MVCC key. Take advantage of this to // sub-slice the raw key directly, instead of calling SplitMVCCKey. p.curUnsafeKey.Key = p.curRawKey[:len(p.curUnsafeKey.Key)] diff --git a/pkg/storage/pebble_mvcc_scanner_test.go b/pkg/storage/pebble_mvcc_scanner_test.go index 597eb9a21c7a..93ca0bf53890 100644 --- a/pkg/storage/pebble_mvcc_scanner_test.go +++ b/pkg/storage/pebble_mvcc_scanner_test.go @@ -42,9 +42,10 @@ func TestMVCCScanWithManyVersionsAndSeparatedIntents(t *testing.T) { keys := []roachpb.Key{roachpb.Key("a"), roachpb.Key("b"), roachpb.Key("c")} // Many versions of each key. for i := 1; i < 10; i++ { + mvccValue := MVCCValue{Value: roachpb.MakeValueFromString(fmt.Sprintf("%d", i))} for _, k := range keys { - require.NoError(t, eng.PutMVCC(MVCCKey{Key: k, Timestamp: hlc.Timestamp{WallTime: int64(i)}}, - []byte(fmt.Sprintf("%d", i)))) + mvccKey := MVCCKey{Key: k, Timestamp: hlc.Timestamp{WallTime: int64(i)}} + require.NoError(t, eng.PutMVCC(mvccKey, mvccValue)) } } // Write a separated lock for the latest version of each key, to make it provisional. @@ -114,7 +115,7 @@ func TestMVCCScanWithManyVersionsAndSeparatedIntents(t *testing.T) { expectedKVs := make([]kv, len(keys)) for i := range expectedKVs { expectedKVs[i].k = MVCCKey{Key: keys[i], Timestamp: hlc.Timestamp{WallTime: 2}} - expectedKVs[i].v = []byte("2") + expectedKVs[i].v = roachpb.MakeValueFromString("2").RawBytes } require.Equal(t, expectedKVs, kvs) } @@ -129,13 +130,13 @@ func TestMVCCScanWithLargeKeyValue(t *testing.T) { largeValue := bytes.Repeat([]byte("l"), 150<<20) // Alternate small and large values. require.NoError(t, eng.PutMVCC(MVCCKey{Key: keys[0], Timestamp: hlc.Timestamp{WallTime: 1}}, - []byte("a"))) + MVCCValue{Value: roachpb.MakeValueFromBytes([]byte("a"))})) require.NoError(t, eng.PutMVCC(MVCCKey{Key: keys[1], Timestamp: hlc.Timestamp{WallTime: 1}}, - largeValue)) + MVCCValue{Value: roachpb.MakeValueFromBytes(largeValue)})) require.NoError(t, eng.PutMVCC(MVCCKey{Key: keys[2], Timestamp: hlc.Timestamp{WallTime: 1}}, - []byte("c"))) + MVCCValue{Value: roachpb.MakeValueFromBytes([]byte("c"))})) require.NoError(t, eng.PutMVCC(MVCCKey{Key: keys[3], Timestamp: hlc.Timestamp{WallTime: 1}}, - largeValue)) + MVCCValue{Value: roachpb.MakeValueFromBytes(largeValue)})) reader := eng.NewReadOnly(StandardDurability) defer reader.Close() @@ -159,14 +160,14 @@ func TestMVCCScanWithLargeKeyValue(t *testing.T) { numKeys := mvccScanner.results.count require.Equal(t, 4, int(numKeys)) require.Equal(t, 4, len(kvData)) - require.Equal(t, 20, len(kvData[0])) + require.Equal(t, 25, len(kvData[0])) require.Equal(t, 32, cap(kvData[0])) - require.Equal(t, 157286419, len(kvData[1])) - require.Equal(t, 157286419, cap(kvData[1])) - require.Equal(t, 20, len(kvData[2])) + require.Equal(t, 157286424, len(kvData[1])) + require.Equal(t, 157286424, cap(kvData[1])) + require.Equal(t, 25, len(kvData[2])) require.Equal(t, 32, cap(kvData[2])) - require.Equal(t, 157286419, len(kvData[3])) - require.Equal(t, 157286419, cap(kvData[3])) + require.Equal(t, 157286424, len(kvData[3])) + require.Equal(t, 157286424, cap(kvData[3])) } func scannerWithAccount( diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index c1e95b8a93d7..0fcf6fac653e 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -165,8 +165,9 @@ func TestPebbleIterReuse(t *testing.T) { batch := eng.NewBatch() defer batch.Close() for i := 0; i < 100; i++ { - key := MVCCKey{[]byte{byte(i)}, hlc.Timestamp{WallTime: 100}} - if err := batch.PutMVCC(key, []byte("foo")); err != nil { + key := MVCCKey{Key: []byte{byte(i)}, Timestamp: hlc.Timestamp{WallTime: 100}} + value := MVCCValue{Value: roachpb.MakeValueFromString("foo")} + if err := batch.PutMVCC(key, value); err != nil { t.Fatal(err) } } @@ -496,8 +497,9 @@ func TestPebbleIterConsistency(t *testing.T) { defer eng.Close() ts1 := hlc.Timestamp{WallTime: 1} ts2 := hlc.Timestamp{WallTime: 2} - k1 := MVCCKey{[]byte("a"), ts1} - require.NoError(t, eng.PutMVCC(k1, []byte("a1"))) + k1 := MVCCKey{Key: []byte("a"), Timestamp: ts1} + v1 := MVCCValue{Value: roachpb.MakeValueFromString("a1")} + require.NoError(t, eng.PutMVCC(k1, v1)) var ( roEngine = eng.NewReadOnly(StandardDurability) @@ -527,7 +529,9 @@ func TestPebbleIterConsistency(t *testing.T) { require.Nil(t, batch2.PinEngineStateForIterators()) // Write a newer version of "a" - require.NoError(t, eng.PutMVCC(MVCCKey{[]byte("a"), ts2}, []byte("a2"))) + k2 := MVCCKey{Key: []byte("a"), Timestamp: ts2} + v2 := MVCCValue{Value: roachpb.MakeValueFromString("a2")} + require.NoError(t, eng.PutMVCC(k2, v2)) checkMVCCIter := func(iter MVCCIterator) { defer iter.Close() @@ -594,8 +598,14 @@ func TestPebbleIterConsistency(t *testing.T) { // The eng iterator will see both values. checkIterSeesBothValues(eng.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: []byte("b")})) // The indexed batches will see 2 values since the second one is written to the batch. - require.NoError(t, batch.PutMVCC(MVCCKey{[]byte("a"), ts2}, []byte("a2"))) - require.NoError(t, batch2.PutMVCC(MVCCKey{[]byte("a"), ts2}, []byte("a2"))) + require.NoError(t, batch.PutMVCC( + MVCCKey{Key: []byte("a"), Timestamp: ts2}, + MVCCValue{Value: roachpb.MakeValueFromString("a2")}, + )) + require.NoError(t, batch2.PutMVCC( + MVCCKey{Key: []byte("a"), Timestamp: ts2}, + MVCCValue{Value: roachpb.MakeValueFromString("a2")}, + )) checkIterSeesBothValues(batch.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: []byte("b")})) checkIterSeesBothValues(batch2.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: []byte("b")})) } @@ -881,7 +891,10 @@ func TestPebbleBackgroundError(t *testing.T) { require.NoError(t, err) defer eng.Close() - require.NoError(t, eng.PutMVCC(MVCCKey{[]byte("a"), hlc.Timestamp{WallTime: 1}}, []byte("a"))) + require.NoError(t, eng.PutMVCC( + MVCCKey{Key: []byte("a"), Timestamp: hlc.Timestamp{WallTime: 1}}, + MVCCValue{Value: roachpb.MakeValueFromString("a")}, + )) require.NoError(t, eng.db.Flush()) } @@ -950,7 +963,8 @@ func generateData(t *testing.T, engine Engine, limits dataLimits, totalEntries i if rng.Float64() < limits.tombstoneChance { size = 0 } - require.NoError(t, engine.PutMVCC(MVCCKey{Key: key, Timestamp: timestamp}, randutil.RandBytes(rng, size)), "Write data to test storage") + value := MVCCValue{Value: roachpb.MakeValueFromBytes(randutil.RandBytes(rng, size))} + require.NoError(t, engine.PutMVCC(MVCCKey{Key: key, Timestamp: timestamp}, value), "Write data to test storage") } require.NoError(t, engine.Flush(), "Flush engine data") } @@ -1174,8 +1188,8 @@ func TestPebbleMVCCTimeIntervalCollectorAndFilter(t *testing.T) { aKey := roachpb.Key("a") for i := 0; i < 10; i++ { require.NoError(t, eng.PutMVCC( - MVCCKey{aKey, hlc.Timestamp{WallTime: int64(i), Logical: 1}}, - []byte(fmt.Sprintf("val%d", i)))) + MVCCKey{Key: aKey, Timestamp: hlc.Timestamp{WallTime: int64(i), Logical: 1}}, + MVCCValue{Value: roachpb.MakeValueFromString(fmt.Sprintf("val%d", i))})) } require.NoError(t, eng.Flush()) iter := eng.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ @@ -1205,9 +1219,10 @@ func TestPebbleFlushCallbackAndDurabilityRequirement(t *testing.T) { defer eng.Close() ts := hlc.Timestamp{WallTime: 1} - k := MVCCKey{[]byte("a"), ts} + k := MVCCKey{Key: []byte("a"), Timestamp: ts} + v := MVCCValue{Value: roachpb.MakeValueFromString("a1")} // Write. - require.NoError(t, eng.PutMVCC(k, []byte("a1"))) + require.NoError(t, eng.PutMVCC(k, v)) cbCount := int32(0) eng.RegisterFlushCompletedCallback(func() { atomic.AddInt32(&cbCount, 1) @@ -1234,7 +1249,7 @@ func TestPebbleFlushCallbackAndDurabilityRequirement(t *testing.T) { } return v } - require.Equal(t, "a1", string(checkGetAndIter(roStandard))) + require.Equal(t, v.Value.RawBytes, checkGetAndIter(roStandard)) // Write is not visible yet. require.Nil(t, checkGetAndIter(roGuaranteed)) require.Nil(t, checkGetAndIter(roGuaranteedPinned)) @@ -1251,5 +1266,5 @@ func TestPebbleFlushCallbackAndDurabilityRequirement(t *testing.T) { // due to iterator caching. roGuaranteed2 := eng.NewReadOnly(GuaranteedDurability) defer roGuaranteed2.Close() - require.Equal(t, "a1", string(checkGetAndIter(roGuaranteed2))) + require.Equal(t, v.Value.RawBytes, checkGetAndIter(roGuaranteed2)) } diff --git a/pkg/storage/sst.go b/pkg/storage/sst.go index b75253c4a066..35672f4b1fca 100644 --- a/pkg/storage/sst.go +++ b/pkg/storage/sst.go @@ -277,7 +277,7 @@ func UpdateSSTTimestamps( return nil, errors.Errorf("unexpected timestamp %s (expected %s) for key %s", key.Timestamp, from, key.Key) } - err = writer.PutMVCC(MVCCKey{Key: iter.UnsafeKey().Key, Timestamp: to}, iter.UnsafeValue()) + err = writer.PutRawMVCC(MVCCKey{Key: key.Key, Timestamp: to}, iter.UnsafeValue()) if err != nil { return nil, err } diff --git a/pkg/storage/sst_test.go b/pkg/storage/sst_test.go index fc467713ce3f..25fe889d27f1 100644 --- a/pkg/storage/sst_test.go +++ b/pkg/storage/sst_test.go @@ -71,7 +71,9 @@ func TestCheckSSTConflictsMaxIntents(t *testing.T) { // Write some committed keys and intents at txn1TS. batch := engine.NewBatch() for _, key := range keys { - require.NoError(t, batch.PutMVCC(MVCCKey{Key: roachpb.Key(key), Timestamp: txn1TS}, []byte("value"))) + mvccKey := MVCCKey{Key: roachpb.Key(key), Timestamp: txn1TS} + mvccValue := MVCCValue{Value: roachpb.MakeValueFromString("value")} + require.NoError(t, batch.PutMVCC(mvccKey, mvccValue)) } for _, key := range intents { require.NoError(t, MVCCPut(ctx, batch, nil, roachpb.Key(key), txn1TS, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("intent"), txn1)) @@ -146,11 +148,11 @@ func BenchmarkUpdateSSTTimestamps(b *testing.B) { b.Fatalf("unknown value mode %d", valueMode) } - var v roachpb.Value - v.SetBytes(value) - v.InitChecksum(key) + var v MVCCValue + v.Value.SetBytes(value) + v.Value.InitChecksum(key) - require.NoError(b, writer.PutMVCC(MVCCKey{Key: key, Timestamp: sstTimestamp}, v.RawBytes)) + require.NoError(b, writer.PutMVCC(MVCCKey{Key: key, Timestamp: sstTimestamp}, v)) } writer.Close() b.Logf("%vMB %v keys", sstFile.Len()/1e6, i) diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 0cff9a55897a..a7dac4766068 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -165,10 +165,25 @@ func (fw *SSTWriter) Put(key MVCCKey, value []byte) error { // An error is returned if it is not greater than any previously added entry // (according to the comparator configured during writer creation). `Close` // cannot have been called. -func (fw *SSTWriter) PutMVCC(key MVCCKey, value []byte) error { +func (fw *SSTWriter) PutMVCC(key MVCCKey, value MVCCValue) error { if key.Timestamp.IsEmpty() { panic("PutMVCC timestamp is empty") } + encValue, err := EncodeMVCCValue(value) + if err != nil { + return err + } + return fw.put(key, encValue) +} + +// PutRawMVCC implements the Writer interface. +// An error is returned if it is not greater than any previously added entry +// (according to the comparator configured during writer creation). `Close` +// cannot have been called. +func (fw *SSTWriter) PutRawMVCC(key MVCCKey, value []byte) error { + if key.Timestamp.IsEmpty() { + panic("PutRawMVCC timestamp is empty") + } return fw.put(key, value) } diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated index 065a36bdebbe..7ad2343a69dd 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated +++ b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated @@ -51,7 +51,7 @@ with t=A txn_remove ---- >> at end: -data: "k"/124.000000000,0 -> /BYTES/v3 +data: "k"/124.000000000,0 -> vheader{ localTs=123.000000000,0 } /BYTES/v3 # Write value4 with an old timestamp without txn...should get a write # too old error. @@ -61,7 +61,7 @@ cput k=k v=v4 cond=v3 ts=123 ---- >> at end: data: "k"/124.000000000,1 -> /BYTES/v4 -data: "k"/124.000000000,0 -> /BYTES/v3 +data: "k"/124.000000000,0 -> vheader{ localTs=123.000000000,0 } /BYTES/v3 error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 123.000000000,0 too old; wrote at 124.000000000,1 # Reset for next test diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums index 00173382f477..7bdadbd56c8c 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums @@ -161,12 +161,12 @@ with t=A meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "k" -> /BYTES/b @11.000000000,0 get: "k" -> /BYTES/b @11.000000000,0 -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=19 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false get: "k" -> /BYTES/b @14.000000000,0 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=32} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false -data: "k"/14.000000000,0 -> /BYTES/b +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=19 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -174,7 +174,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 -# Ensure that the deleted value doens't surface. Instead, if we ignore the +# Ensure that the deleted value doesn't surface. Instead, if we ignore the # now-newest seq, we get the write before it. run ok @@ -225,11 +225,11 @@ with t=B txn_begin ts=20 get k=k ---- -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=19 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false get: "k" -> /BYTES/b @14.000000000,0 >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -255,7 +255,7 @@ meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 mi get: "l" -> /BYTES/c @20.000000000,0 >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -282,7 +282,7 @@ meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 mi get: "l" -> >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=35} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -320,7 +320,7 @@ meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 mi get: "m" -> /BYTES/c @30.000000000,0 >> at end: txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=30.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -342,7 +342,7 @@ meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 mi get: "m" -> /BYTES/a @30.000000000,0 >> at end: txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=30.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -370,7 +370,7 @@ meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 mi get: "n" -> /BYTES/c @40.000000000,0 >> at end: txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=40.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -394,11 +394,11 @@ with t=D ---- meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "n" -> /BYTES/c @40.000000000,0 -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=20 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "n" -> /BYTES/c @45.000000000,0 >> at end: txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=40.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -406,8 +406,8 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false -data: "n"/45.000000000,0 -> /BYTES/c +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=20 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c # Ignore sequence numbers other than the current one, then commit. The value # shouldn't change. @@ -421,12 +421,12 @@ with t=E txn_begin ts=50 get k=n ---- -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=20 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "n" -> /BYTES/c @45.000000000,0 get: "n" -> /BYTES/c @45.000000000,0 >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -434,7 +434,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c # Write a couple values at different sequence numbers on this key, then ignore # them all, then do a resolve_intent while the txn is pending. The intent should @@ -458,7 +458,7 @@ get: "n" -> /BYTES/c @45.000000000,0 get: "o" -> >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -468,7 +468,7 @@ data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "n"/50.000000000,0 -> /BYTES/c -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/c @@ -485,7 +485,7 @@ get: "n" -> /BYTES/c @45.000000000,0 get: "o" -> >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -493,7 +493,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c run ok with t=E @@ -533,7 +533,7 @@ with t=E meta: "o" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -541,7 +541,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/c @@ -556,7 +556,7 @@ with t=E get: "o" -> >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=55.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -564,7 +564,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c run error @@ -589,7 +589,7 @@ meta: "o" -> txn={id=00000000 key="o" pri=0.00000000 epo=0 ts=50.000000000,0 min get: "o" -> /BYTES/b @50.000000000,0 >> at end: txn: "F" meta={id=00000000 key="o" pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=40.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -597,7 +597,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c meta: "o"/0,0 -> txn={id=00000000 key="o" pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/b @@ -618,7 +618,7 @@ meta: "o" -> txn={id=00000000 key="o" pri=0.00000000 epo=0 ts=50.000000000,0 min get: "o" -> /BYTES/a @50.000000000,0 >> at end: txn: "F" meta={id=00000000 key="o" pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=40.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -626,6 +626,6 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c meta: "o"/0,0 -> txn={id=00000000 key="o" pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=10} ts=50.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/a diff --git a/pkg/storage/testdata/mvcc_histories/local_timestamp b/pkg/storage/testdata/mvcc_histories/local_timestamp new file mode 100644 index 000000000000..1f2670f66d1e --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/local_timestamp @@ -0,0 +1,324 @@ +# Exercise each write operation with a local timestamp. + +run ok +with ts=20 + put localTs=10 k=k1 v=v + put localTs=20 k=k2 v=v + put localTs=30 k=k3 v=v +---- +>> at end: +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/20.000000000,0 -> /BYTES/v + +run ok +with ts=30 + cput localTs=40 k=k1 v=v cond=v + cput localTs=30 k=k2 v=v cond=v + cput localTs=20 k=k3 v=v cond=v +---- +>> at end: +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +data: "k2"/30.000000000,0 -> /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v +data: "k3"/20.000000000,0 -> /BYTES/v + +run ok +with ts=40 + del localTs=30 k=k1 + del localTs=40 k=k2 + del localTs=50 k=k3 +---- +>> at end: +data: "k1"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } / +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +data: "k2"/40.000000000,0 -> / +data: "k2"/30.000000000,0 -> /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/40.000000000,0 -> / +data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v +data: "k3"/20.000000000,0 -> /BYTES/v + +run ok +with ts=50 + increment localTs=60 k=k1 + increment localTs=50 k=k2 + increment localTs=40 k=k3 +---- +inc: current value = 1 +inc: current value = 1 +inc: current value = 1 +>> at end: +data: "k1"/50.000000000,0 -> /INT/1 +data: "k1"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } / +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +data: "k2"/50.000000000,0 -> /INT/1 +data: "k2"/40.000000000,0 -> / +data: "k2"/30.000000000,0 -> /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/50.000000000,0 -> vheader{ localTs=40.000000000,0 } /INT/1 +data: "k3"/40.000000000,0 -> / +data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v +data: "k3"/20.000000000,0 -> /BYTES/v + +run ok +with ts=60 + del_range localTs=50 k=k1 end=k2 + del_range localTs=60 k=k2 end=k3 + del_range localTs=70 k=k3 end=k4 +---- +del_range: "k1"-"k2" -> deleted 1 key(s) +del_range: "k2"-"k3" -> deleted 1 key(s) +del_range: "k3"-"k4" -> deleted 1 key(s) +>> at end: +data: "k1"/60.000000000,0 -> vheader{ localTs=50.000000000,0 } / +data: "k1"/50.000000000,0 -> /INT/1 +data: "k1"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } / +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +data: "k2"/60.000000000,0 -> / +data: "k2"/50.000000000,0 -> /INT/1 +data: "k2"/40.000000000,0 -> / +data: "k2"/30.000000000,0 -> /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/60.000000000,0 -> / +data: "k3"/50.000000000,0 -> vheader{ localTs=40.000000000,0 } /INT/1 +data: "k3"/40.000000000,0 -> / +data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v +data: "k3"/20.000000000,0 -> /BYTES/v + +run ok +with ts=19 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> +get: "k2" -> +get: "k3" -> + +run ok +with ts=20 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /BYTES/v @20.000000000,0 +get: "k2" -> /BYTES/v @20.000000000,0 +get: "k3" -> /BYTES/v @20.000000000,0 + +run ok +with ts=29 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /BYTES/v @20.000000000,0 +get: "k2" -> /BYTES/v @20.000000000,0 +get: "k3" -> /BYTES/v @20.000000000,0 + +run ok +with ts=30 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /BYTES/v @30.000000000,0 +get: "k2" -> /BYTES/v @30.000000000,0 +get: "k3" -> /BYTES/v @30.000000000,0 + +run ok +with ts=39 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /BYTES/v @30.000000000,0 +get: "k2" -> /BYTES/v @30.000000000,0 +get: "k3" -> /BYTES/v @30.000000000,0 + +run ok +with ts=40 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> +get: "k2" -> +get: "k3" -> + +run ok +with ts=49 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> +get: "k2" -> +get: "k3" -> + +run ok +with ts=50 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /INT/1 @50.000000000,0 +get: "k2" -> /INT/1 @50.000000000,0 +get: "k3" -> /INT/1 @50.000000000,0 + +run ok +with ts=59 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /INT/1 @50.000000000,0 +get: "k2" -> /INT/1 @50.000000000,0 +get: "k3" -> /INT/1 @50.000000000,0 + +run ok +with ts=60 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> +get: "k2" -> +get: "k3" -> + +run ok +clear_range k=k1 end=k4 +---- +>> at end: + + + +# Exercise intent resolution with and without prior observation timestamps. + +run ok +with t=A ts=20 localTs=10 + txn_begin + put k=k1 v=v + put k=k2 v=v + put k=k3 v=v + put k=k4 v=v + put k=k5 v=v + put k=k6 v=v + put k=k7 v=v + put k=k8 v=v + put k=k9 v=v + put k=k10 v=v + put k=k11 v=v + put k=k12 v=v +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k10"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k11"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k11"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k12"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k12"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k2"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k4"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k5"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k6"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k7"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k8"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k8"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k9"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k9"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v + +run ok +with t=A + txn_step + txn_advance ts=30 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 + +run ok +with t=A localTs=20 + put k=k1 v=v2 + put k=k2 v=v2 + put k=k3 v=v2 + put k=k4 v=v2 + put k=k5 v=v2 + put k=k6 v=v2 + put k=k7 v=v2 + put k=k8 v=v2 + put k=k9 v=v2 + put k=k10 v=v2 + put k=k11 v=v2 + put k=k12 v=v2 +---- +>> at end: +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k1"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k10"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k11"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k11"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k12"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k12"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k2"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k4"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k4"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k5"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k5"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k6"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k6"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k7"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k7"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k8"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k8"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k9"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k9"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 + +run ok +with t=A + txn_advance ts=40 + resolve_intent k=k1 status=ABORTED + resolve_intent k=k2 status=ABORTED clockWhilePending=20 + resolve_intent k=k3 status=ABORTED clockWhilePending=30 + resolve_intent k=k4 status=ABORTED clockWhilePending=40 + resolve_intent k=k5 status=PENDING + resolve_intent k=k6 status=PENDING clockWhilePending=20 + resolve_intent k=k7 status=PENDING clockWhilePending=30 + resolve_intent k=k8 status=PENDING clockWhilePending=40 + resolve_intent k=k9 status=COMMITTED + resolve_intent k=k10 status=COMMITTED clockWhilePending=20 + resolve_intent k=k11 status=COMMITTED clockWhilePending=30 + resolve_intent k=k12 status=COMMITTED clockWhilePending=40 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k10"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +data: "k11"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } /BYTES/v2 +data: "k12"/40.000000000,0 -> /BYTES/v2 +meta: "k5"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k5"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k6"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k6"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k7"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k7"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } /BYTES/v2 +meta: "k8"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k8"/40.000000000,0 -> /BYTES/v2 +data: "k9"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit index 41f854262a02..94bee39a51b5 100644 --- a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit @@ -4,25 +4,25 @@ # k1: value @ ts 10 # # k2: value @ ts 20 -# k2: value @ ts 10? +# k2: value @ ts 10, localTs 5 # -# k3: value @ ts 20? +# k3: value @ ts 20, localTs 10 # k3: value @ ts 10 # -# k4: value @ ts 20? -# k4: value @ ts 10? +# k4: value @ ts 20, localTs 10 +# k4: value @ ts 10, localTs 5 # # k5: intent @ ts 20 # k5: value @ ts 10 # # k6: intent @ ts 20 -# k6: value @ ts 10? +# k6: value @ ts 10, localTs 5 # -# k7: intent @ ts 20? +# k7: intent @ ts 20, localTs 10 # k7: value @ ts 10 # -# k8: intent @ ts 20? -# k8: value @ ts 10? +# k8: intent @ ts 20, localTs 10 +# k8: value @ ts 10, localTs 5 # run ok @@ -36,42 +36,42 @@ data: "k1"/10.000000000,0 -> /BYTES/v1 run ok with k=k2 - put v=v3 ts=10,0? + put v=v3 ts=10,0 localTs=5,0 put v=v4 ts=20,0 ---- >> at end: data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 run ok with k=k3 put v=v5 ts=10,0 - put v=v6 ts=20,0? + put v=v6 ts=20,0 localTs=10,0 ---- >> at end: data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 run ok with k=k4 - put v=v7 ts=10,0? - put v=v8 ts=20,0? + put v=v7 ts=10,0 localTs=5,0 + put v=v8 ts=20,0 localTs=10,0 ---- >> at end: data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 -data: "k4"/20.000000000,0? -> /BYTES/v8 -data: "k4"/10.000000000,0? -> /BYTES/v7 +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v8 +data: "k4"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v7 run ok with k=k5 @@ -84,18 +84,18 @@ txn: "A" meta={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0 data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 -data: "k4"/20.000000000,0? -> /BYTES/v8 -data: "k4"/10.000000000,0? -> /BYTES/v7 +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v8 +data: "k4"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v7 meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 run ok with k=k6 - put v=v11 ts=10,0? + put v=v11 ts=10,0 localTs=5,0 txn_begin t=B ts=20,0 put t=B v=v12 ---- @@ -104,72 +104,72 @@ txn: "B" meta={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0 data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 -data: "k4"/20.000000000,0? -> /BYTES/v8 -data: "k4"/10.000000000,0? -> /BYTES/v7 +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v8 +data: "k4"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v7 meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k6"/20.000000000,0 -> /BYTES/v12 -data: "k6"/10.000000000,0? -> /BYTES/v11 +data: "k6"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v11 run ok with k=k7 put v=v13 ts=10,0 - txn_begin t=C ts=20,0? - put t=C v=v14 + txn_begin t=C ts=20,0 + put t=C v=v14 localTs=10,0 ---- >> at end: -txn: "C" meta={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0? wto=false gul=0,0 +txn: "C" meta={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 -data: "k4"/20.000000000,0? -> /BYTES/v8 -data: "k4"/10.000000000,0? -> /BYTES/v7 +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v8 +data: "k4"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v7 meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k6"/20.000000000,0 -> /BYTES/v12 -data: "k6"/10.000000000,0? -> /BYTES/v11 -meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true -data: "k7"/20.000000000,0? -> /BYTES/v14 +data: "k6"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=21 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v14 data: "k7"/10.000000000,0 -> /BYTES/v13 run ok with k=k8 - put v=v15 ts=10,0? - txn_begin t=D ts=20,0? - put t=D v=v16 + put v=v15 ts=10,0 localTs=5,0 + txn_begin t=D ts=20,0 + put t=D v=v16 localTs=10,0 ---- >> at end: -txn: "D" meta={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0? wto=false gul=0,0 +txn: "D" meta={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 -data: "k4"/20.000000000,0? -> /BYTES/v8 -data: "k4"/10.000000000,0? -> /BYTES/v7 +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v8 +data: "k4"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v7 meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k6"/20.000000000,0 -> /BYTES/v12 -data: "k6"/10.000000000,0? -> /BYTES/v11 -meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true -data: "k7"/20.000000000,0? -> /BYTES/v14 +data: "k6"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=21 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v14 data: "k7"/10.000000000,0 -> /BYTES/v13 -meta: "k8"/0,0 -> txn={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true -data: "k8"/20.000000000,0? -> /BYTES/v16 -data: "k8"/10.000000000,0? -> /BYTES/v15 +meta: "k8"/0,0 -> txn={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=21 mergeTs= txnDidNotUpdateMeta=true +data: "k8"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v16 +data: "k8"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v15 # Test cases: # @@ -204,13 +204,13 @@ run error get t=txn1 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k3 localUncertaintyLimit=5,0 @@ -226,13 +226,13 @@ run error get t=txn1 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k5 localUncertaintyLimit=5,0 @@ -248,13 +248,13 @@ run error get t=txn1 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k7 localUncertaintyLimit=5,0 @@ -270,13 +270,13 @@ run error get t=txn1 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok @@ -299,13 +299,13 @@ run error get t=txn2 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k3 localUncertaintyLimit=5,0 @@ -321,13 +321,13 @@ run error get t=txn2 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k5 localUncertaintyLimit=5,0 @@ -343,13 +343,13 @@ run error get t=txn2 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k7 localUncertaintyLimit=5,0 @@ -365,13 +365,13 @@ run error get t=txn2 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok @@ -394,37 +394,35 @@ run error get t=txn3 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] -run error +run ok get t=txn3 k=k3 localUncertaintyLimit=5,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] -run error +run ok scan t=txn3 k=k3 localUncertaintyLimit=5,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn3 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn3 k=k5 localUncertaintyLimit=5,0 @@ -440,37 +438,35 @@ run error get t=txn3 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] -run error +run ok get t=txn3 k=k7 localUncertaintyLimit=5,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] -run error +run ok scan t=txn3 k=k7 localUncertaintyLimit=5,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn3 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -493,37 +489,35 @@ run error get t=txn4 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] -run error +run ok get t=txn4 k=k3 localUncertaintyLimit=5,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] -run error +run ok scan t=txn4 k=k3 localUncertaintyLimit=5,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn4 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn4 k=k5 localUncertaintyLimit=5,0 @@ -539,37 +533,35 @@ run error get t=txn4 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] -run error +run ok get t=txn4 k=k7 localUncertaintyLimit=5,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] -run error +run ok scan t=txn4 k=k7 localUncertaintyLimit=5,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn4 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -594,13 +586,13 @@ run error get t=txn5 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k3 localUncertaintyLimit=10,0 @@ -618,13 +610,13 @@ run error get t=txn5 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k5 localUncertaintyLimit=10,0 @@ -642,13 +634,13 @@ run error get t=txn5 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k7 localUncertaintyLimit=10,0 @@ -666,13 +658,13 @@ run error get t=txn5 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok @@ -697,37 +689,37 @@ run error get t=txn6 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k5 localUncertaintyLimit=10,0 @@ -745,37 +737,37 @@ run error get t=txn6 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -800,37 +792,37 @@ run error get t=txn7 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k5 localUncertaintyLimit=10,0 @@ -848,37 +840,37 @@ run error get t=txn7 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -903,37 +895,37 @@ run error get t=txn8 k=k2 localUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k2 localUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k5 localUncertaintyLimit=15,0 @@ -951,37 +943,37 @@ run error get t=txn8 k=k6 localUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k6 localUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1006,37 +998,37 @@ run error get t=txn9 k=k2 localUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k2 localUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k5 localUncertaintyLimit=15,0 @@ -1054,37 +1046,37 @@ run error get t=txn9 k=k6 localUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k6 localUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1121,25 +1113,25 @@ run error get t=txn10 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k5 localUncertaintyLimit=20,0 @@ -1169,25 +1161,25 @@ run error get t=txn10 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1209,12 +1201,12 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn11 k=k2 localUncertaintyLimit=10,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn11 k=k2 localUncertaintyLimit=10,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run ok get t=txn11 k=k3 localUncertaintyLimit=10,0 @@ -1229,12 +1221,12 @@ scan: "k3" -> /BYTES/v5 @10.000000000,0 run ok get t=txn11 k=k4 localUncertaintyLimit=10,0 ---- -get: "k4" -> /BYTES/v7 @10.000000000,0? +get: "k4" -> /BYTES/v7 @10.000000000,0 run ok scan t=txn11 k=k4 localUncertaintyLimit=10,0 ---- -scan: "k4" -> /BYTES/v7 @10.000000000,0? +scan: "k4" -> /BYTES/v7 @10.000000000,0 run ok get t=txn11 k=k5 localUncertaintyLimit=10,0 @@ -1249,12 +1241,12 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn11 k=k6 localUncertaintyLimit=10,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn11 k=k6 localUncertaintyLimit=10,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run ok get t=txn11 k=k7 localUncertaintyLimit=10,0 @@ -1269,12 +1261,12 @@ scan: "k7" -> /BYTES/v13 @10.000000000,0 run ok get t=txn11 k=k8 localUncertaintyLimit=10,0 ---- -get: "k8" -> /BYTES/v15 @10.000000000,0? +get: "k8" -> /BYTES/v15 @10.000000000,0 run ok scan t=txn11 k=k8 localUncertaintyLimit=10,0 ---- -scan: "k8" -> /BYTES/v15 @10.000000000,0? +scan: "k8" -> /BYTES/v15 @10.000000000,0 run ok @@ -1296,36 +1288,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn12 k=k2 localUncertaintyLimit=10,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn12 k=k2 localUncertaintyLimit=10,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn12 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn12 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn12 k=k5 localUncertaintyLimit=10,0 @@ -1340,36 +1332,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn12 k=k6 localUncertaintyLimit=10,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn12 k=k6 localUncertaintyLimit=10,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn12 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn12 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1391,36 +1383,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn13 k=k2 localUncertaintyLimit=10,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn13 k=k2 localUncertaintyLimit=10,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn13 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn13 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn13 k=k5 localUncertaintyLimit=10,0 @@ -1435,36 +1427,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn13 k=k6 localUncertaintyLimit=10,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn13 k=k6 localUncertaintyLimit=10,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn13 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn13 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1486,36 +1478,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn14 k=k2 localUncertaintyLimit=15,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn14 k=k2 localUncertaintyLimit=15,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn14 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn14 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn14 k=k5 localUncertaintyLimit=15,0 @@ -1530,36 +1522,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn14 k=k6 localUncertaintyLimit=15,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn14 k=k6 localUncertaintyLimit=15,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn14 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn14 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1581,36 +1573,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn15 k=k2 localUncertaintyLimit=15,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn15 k=k2 localUncertaintyLimit=15,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn15 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn15 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn15 k=k5 localUncertaintyLimit=15,0 @@ -1625,36 +1617,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn15 k=k6 localUncertaintyLimit=15,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn15 k=k6 localUncertaintyLimit=15,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn15 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn15 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1691,25 +1683,25 @@ run error get t=txn16 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k5 localUncertaintyLimit=20,0 @@ -1739,25 +1731,25 @@ run error get t=txn16 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1779,36 +1771,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn17 k=k2 localUncertaintyLimit=15,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn17 k=k2 localUncertaintyLimit=15,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn17 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn17 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn17 k=k5 localUncertaintyLimit=15,0 @@ -1823,36 +1815,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn17 k=k6 localUncertaintyLimit=15,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn17 k=k6 localUncertaintyLimit=15,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn17 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn17 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1874,36 +1866,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn18 k=k2 localUncertaintyLimit=15,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn18 k=k2 localUncertaintyLimit=15,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn18 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn18 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn18 k=k5 localUncertaintyLimit=15,0 @@ -1918,36 +1910,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn18 k=k6 localUncertaintyLimit=15,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn18 k=k6 localUncertaintyLimit=15,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn18 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn18 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1984,25 +1976,25 @@ run error get t=txn19 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k5 localUncertaintyLimit=20,0 @@ -2032,25 +2024,25 @@ run error get t=txn19 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -2082,22 +2074,22 @@ scan: "k2" -> /BYTES/v4 @20.000000000,0 run ok get t=txn20 k=k3 localUncertaintyLimit=20,0 ---- -get: "k3" -> /BYTES/v6 @20.000000000,0? +get: "k3" -> /BYTES/v6 @20.000000000,0 run ok scan t=txn20 k=k3 localUncertaintyLimit=20,0 ---- -scan: "k3" -> /BYTES/v6 @20.000000000,0? +scan: "k3" -> /BYTES/v6 @20.000000000,0 run ok get t=txn20 k=k4 localUncertaintyLimit=20,0 ---- -get: "k4" -> /BYTES/v8 @20.000000000,0? +get: "k4" -> /BYTES/v8 @20.000000000,0 run ok scan t=txn20 k=k4 localUncertaintyLimit=20,0 ---- -scan: "k4" -> /BYTES/v8 @20.000000000,0? +scan: "k4" -> /BYTES/v8 @20.000000000,0 run error get t=txn20 k=k5 localUncertaintyLimit=20,0 @@ -2254,13 +2246,13 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2276,13 +2268,13 @@ run error get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2298,13 +2290,13 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2320,13 +2312,13 @@ run error get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 @@ -2342,37 +2334,35 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] -run error +run ok get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] -run error +run ok scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 @@ -2388,37 +2378,35 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] -run error +run ok get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] -run error +run ok scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2436,13 +2424,13 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2460,13 +2448,13 @@ run error get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2484,13 +2472,13 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2508,13 +2496,13 @@ run error get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 @@ -2532,37 +2520,37 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 @@ -2580,37 +2568,37 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2640,25 +2628,25 @@ run error get k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2688,25 +2676,25 @@ run error get k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2721,12 +2709,12 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run ok get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2741,12 +2729,12 @@ scan: "k3" -> /BYTES/v5 @10.000000000,0 run ok get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -get: "k4" -> /BYTES/v7 @10.000000000,0? +get: "k4" -> /BYTES/v7 @10.000000000,0 run ok scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -scan: "k4" -> /BYTES/v7 @10.000000000,0? +scan: "k4" -> /BYTES/v7 @10.000000000,0 run ok get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2761,12 +2749,12 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run ok get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2781,12 +2769,12 @@ scan: "k7" -> /BYTES/v13 @10.000000000,0 run ok get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -get: "k8" -> /BYTES/v15 @10.000000000,0? +get: "k8" -> /BYTES/v15 @10.000000000,0 run ok scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -scan: "k8" -> /BYTES/v15 @10.000000000,0? +scan: "k8" -> /BYTES/v15 @10.000000000,0 run ok get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 @@ -2801,36 +2789,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 @@ -2845,36 +2833,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2904,25 +2892,25 @@ run error get k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2952,25 +2940,25 @@ run error get k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2995,22 +2983,22 @@ scan: "k2" -> /BYTES/v4 @20.000000000,0 run ok get k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- -get: "k3" -> /BYTES/v6 @20.000000000,0? +get: "k3" -> /BYTES/v6 @20.000000000,0 run ok scan k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- -scan: "k3" -> /BYTES/v6 @20.000000000,0? +scan: "k3" -> /BYTES/v6 @20.000000000,0 run ok get k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- -get: "k4" -> /BYTES/v8 @20.000000000,0? +get: "k4" -> /BYTES/v8 @20.000000000,0 run ok scan k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- -scan: "k4" -> /BYTES/v8 @20.000000000,0? +scan: "k4" -> /BYTES/v8 @20.000000000,0 run error get k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps new file mode 100644 index 000000000000..41f854262a02 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps @@ -0,0 +1,3061 @@ +# Setup: +# +# k1: value @ ts 20 +# k1: value @ ts 10 +# +# k2: value @ ts 20 +# k2: value @ ts 10? +# +# k3: value @ ts 20? +# k3: value @ ts 10 +# +# k4: value @ ts 20? +# k4: value @ ts 10? +# +# k5: intent @ ts 20 +# k5: value @ ts 10 +# +# k6: intent @ ts 20 +# k6: value @ ts 10? +# +# k7: intent @ ts 20? +# k7: value @ ts 10 +# +# k8: intent @ ts 20? +# k8: value @ ts 10? +# + +run ok +with k=k1 + put v=v1 ts=10,0 + put v=v2 ts=20,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 + +run ok +with k=k2 + put v=v3 ts=10,0? + put v=v4 ts=20,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 + +run ok +with k=k3 + put v=v5 ts=10,0 + put v=v6 ts=20,0? +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 + +run ok +with k=k4 + put v=v7 ts=10,0? + put v=v8 ts=20,0? +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0? -> /BYTES/v8 +data: "k4"/10.000000000,0? -> /BYTES/v7 + +run ok +with k=k5 + put v=v9 ts=10,0 + txn_begin t=A ts=20,0 + put t=A v=v10 +---- +>> at end: +txn: "A" meta={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0? -> /BYTES/v8 +data: "k4"/10.000000000,0? -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 + +run ok +with k=k6 + put v=v11 ts=10,0? + txn_begin t=B ts=20,0 + put t=B v=v12 +---- +>> at end: +txn: "B" meta={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0? -> /BYTES/v8 +data: "k4"/10.000000000,0? -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0? -> /BYTES/v11 + +run ok +with k=k7 + put v=v13 ts=10,0 + txn_begin t=C ts=20,0? + put t=C v=v14 +---- +>> at end: +txn: "C" meta={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0? wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0? -> /BYTES/v8 +data: "k4"/10.000000000,0? -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0? -> /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0? -> /BYTES/v14 +data: "k7"/10.000000000,0 -> /BYTES/v13 + +run ok +with k=k8 + put v=v15 ts=10,0? + txn_begin t=D ts=20,0? + put t=D v=v16 +---- +>> at end: +txn: "D" meta={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0? wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0? -> /BYTES/v8 +data: "k4"/10.000000000,0? -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0? -> /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0? -> /BYTES/v14 +data: "k7"/10.000000000,0 -> /BYTES/v13 +meta: "k8"/0,0 -> txn={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k8"/20.000000000,0? -> /BYTES/v16 +data: "k8"/10.000000000,0? -> /BYTES/v15 + +# Test cases: +# +# for ts in (5, 10, 15, 20, 25): +# for localUncertaintyLimit in (5, 10, 15, 20, 25): +# if localUncertaintyLimit < ts: continue +# for globalUncertaintyLimit in (5, 10, 15, 20, 25): +# if globalUncertaintyLimit < ts: continue +# if globalUncertaintyLimit <= localUncertaintyLimit: continue +# for k in (k1, k2, k3, k4, k5, k6, k7, k8): +# for op in (get, scan): +# testCase() +# + +run ok +txn_begin t=txn1 ts=5,0 globalUncertaintyLimit=10,0 +---- +>> at end: +txn: "txn1" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=10.000000000,0 + +run ok +get t=txn1 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn1 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get t=txn1 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run error +scan t=txn1 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run ok +get t=txn1 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn1 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run error +get t=txn1 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run error +scan t=txn1 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run ok +get t=txn1 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn1 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get t=txn1 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run error +scan t=txn1 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run ok +get t=txn1 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn1 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run error +get t=txn1 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run error +scan t=txn1 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn2 ts=5,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn2" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=15.000000000,0 + +run ok +get t=txn2 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn2 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get t=txn2 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn2 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run ok +get t=txn2 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn2 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run error +get t=txn2 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn2 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run ok +get t=txn2 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn2 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get t=txn2 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn2 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run ok +get t=txn2 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn2 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run error +get t=txn2 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn2 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn3 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn3" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn3 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn3 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get t=txn3 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn3 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn3 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run ok +get t=txn3 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn3 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get t=txn3 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn3 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn3 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn4 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn4" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn4 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn4 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get t=txn4 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn4 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn4 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run ok +get t=txn4 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn4 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get t=txn4 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn4 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn4 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn5 ts=5,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn5" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=15.000000000,0 + +run error +get t=txn5 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn6 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn6" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run error +get t=txn6 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn7 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn7" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn7 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn8 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn8" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run error +get t=txn8 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn9 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn9" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn9 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn10 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn10" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn10 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn11 ts=10,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn11" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=15.000000000,0 + +run ok +get t=txn11 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn11 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn11 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn11 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +get t=txn11 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn11 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn11 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0? + +run ok +scan t=txn11 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0? + +run ok +get t=txn11 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn11 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn11 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn11 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +get t=txn11 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn11 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn11 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0? + +run ok +scan t=txn11 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0? + + +run ok +txn_begin t=txn12 ts=10,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn12" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn12 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn12 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn12 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn12 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn12 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn12 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn12 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn12 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run ok +get t=txn12 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn12 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn12 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn12 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn12 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn12 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn12 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn12 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn13 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn13" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn13 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn13 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn13 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn13 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn13 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn13 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn13 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn13 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run ok +get t=txn13 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn13 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn13 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn13 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn13 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn13 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn13 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn13 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn14 ts=10,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn14" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn14 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn14 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn14 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn14 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn14 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn14 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn14 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn14 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run ok +get t=txn14 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn14 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn14 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn14 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn14 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn14 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn14 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn14 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn15 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn15" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn15 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn15 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn15 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn15 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn15 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn15 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn15 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn15 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run ok +get t=txn15 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn15 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn15 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn15 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn15 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn15 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn15 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn15 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn16 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn16" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn16 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn17 ts=15,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn17" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn17 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn17 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn17 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn17 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn17 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn17 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn17 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn17 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run ok +get t=txn17 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn17 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn17 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn17 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn17 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn17 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn17 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn17 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn18 ts=15,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn18" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn18 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn18 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn18 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn18 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn18 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn18 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn18 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn18 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run ok +get t=txn18 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn18 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn18 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn18 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn18 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn18 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn18 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn18 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn19 ts=15,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn19" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn19 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn20 ts=20,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn20" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn20 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +scan t=txn20 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +get t=txn20 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +scan t=txn20 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +get t=txn20 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> /BYTES/v6 @20.000000000,0? + +run ok +scan t=txn20 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3" -> /BYTES/v6 @20.000000000,0? + +run ok +get t=txn20 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> /BYTES/v8 @20.000000000,0? + +run ok +scan t=txn20 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4" -> /BYTES/v8 @20.000000000,0? + +run error +get t=txn20 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +scan t=txn20 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +get t=txn20 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +scan t=txn20 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +get t=txn20 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +scan t=txn20 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +get t=txn20 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +run error +scan t=txn20 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +# A subset of the previous test cases, but with non-transactional reads: +# +# for ts in (5, 15, 25): +# for localUncertaintyLimit in (5, 15, 25): +# if localUncertaintyLimit < ts: continue +# for globalUncertaintyLimit in (5, 15, 25): +# if globalUncertaintyLimit < ts: continue +# if globalUncertaintyLimit < localUncertaintyLimit: continue +# for k in (k1, k2, k3, k4, k5, k6, k7, k8): +# for op in (get, scan): +# testCase() +# + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> + +run ok +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> + +run error +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> + +run ok +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> + +run error +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0? + +run ok +scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0? + +run ok +get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0? + +run ok +scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0? + +run ok +get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +scan k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +get k=k2 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +scan k=k2 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +get k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> /BYTES/v6 @20.000000000,0? + +run ok +scan k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3" -> /BYTES/v6 @20.000000000,0? + +run ok +get k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> /BYTES/v8 @20.000000000,0? + +run ok +scan k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4" -> /BYTES/v8 @20.000000000,0? + +run error +get k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +scan k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +get k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +scan k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +get k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +scan k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +get k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +run error +scan k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 642d3821fb1b..a337c0004427 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1052,6 +1052,7 @@ func TestLint(t *testing.T) { ":!rpc/codec.go", ":!rpc/codec_test.go", ":!settings/settings_test.go", + ":!storage/mvcc_value.go", ":!sql/types/types_jsonpb.go", ":!sql/schemachanger/scplan/internal/scgraphviz/graphviz.go", ) @@ -1096,6 +1097,7 @@ func TestLint(t *testing.T) { ":!util/hlc/timestamp.go", ":!rpc/codec.go", ":!rpc/codec_test.go", + ":!storage/mvcc_value.go", ":!sql/types/types_jsonpb.go", ) if err != nil { diff --git a/pkg/testutils/sstutil/kv.go b/pkg/testutils/sstutil/kv.go index a01e8b9f0805..ff2b8ecf16b6 100644 --- a/pkg/testutils/sstutil/kv.go +++ b/pkg/testutils/sstutil/kv.go @@ -51,7 +51,7 @@ func (kv KV) Value() roachpb.Value { return value } -// ValueBytes returns the roachpb.Value byte-representation of the value. -func (kv KV) ValueBytes() []byte { - return kv.Value().RawBytes +// MVCCValue returns the storage.MVCCValue representation of the value. +func (kv KV) MVCCValue() storage.MVCCValue { + return storage.MVCCValue{Value: kv.Value()} } diff --git a/pkg/testutils/sstutil/sstutil.go b/pkg/testutils/sstutil/sstutil.go index 8ab97c0e270f..a0e9cb202647 100644 --- a/pkg/testutils/sstutil/sstutil.go +++ b/pkg/testutils/sstutil/sstutil.go @@ -41,12 +41,12 @@ func MakeSST(t *testing.T, st *cluster.Settings, kvs []KV) ([]byte, roachpb.Key, end = kv.Key() } if kv.Timestamp().IsEmpty() { - meta := &enginepb.MVCCMetadata{RawBytes: kv.ValueBytes()} + meta := &enginepb.MVCCMetadata{RawBytes: kv.Value().RawBytes} metaBytes, err := protoutil.Marshal(meta) require.NoError(t, err) require.NoError(t, writer.PutUnversioned(kv.Key(), metaBytes)) } else { - require.NoError(t, writer.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + require.NoError(t, writer.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) } } require.NoError(t, writer.Finish()) @@ -73,8 +73,9 @@ func ScanSST(t *testing.T, sst []byte) []KV { } k := iter.UnsafeKey() - v := roachpb.Value{RawBytes: iter.UnsafeValue()} - value, err := v.GetBytes() + v, err := storage.DecodeMVCCValue(iter.UnsafeValue()) + require.NoError(t, err) + value, err := v.Value.GetBytes() require.NoError(t, err) kvs = append(kvs, KV{ KeyString: string(k.Key), diff --git a/pkg/util/hlc/doc.go b/pkg/util/hlc/doc.go index c6080bdd350a..733cc0cd75f6 100644 --- a/pkg/util/hlc/doc.go +++ b/pkg/util/hlc/doc.go @@ -70,15 +70,13 @@ The linearizability guarantee is important to note as two sequential (in real time) transactions via two different gateway nodes can be assigned timestamps in reverse order (the second gateway's clock may be behind), but must still see results according to real-time order if they access overlapping keys (e.g. B -must see A's write). Also keep in mind that an intent's written timestamp +must see A's write). Also keep in mind that an intent's local timestamp signifies when the intent itself was written, but the final value will be resolved to the transaction's commit timestamp, which may be later than the -written timestamp. Since the commit status and timestamp are non-local +local timestamp. Since the commit status and timestamp are non-local properties, a range may contain committed values (as unresolved intents) that turn out to exist in the future of the local HLC when the intent gets resolved. -TODO(nvanbenschoten): Update the above on written timestamps after #72121. - - Cooperative lease transfers (Raft channel). During a cooperative lease transfer from one replica of a range to another, the outgoing leaseholder revokes its lease before its expiration time and consults its clock to @@ -89,7 +87,7 @@ TODO(nvanbenschoten): Update the above on written timestamps after #72121. The invariant that a leaseholder's clock is always >= its lease's start time is used in a few places. First, it ensures that the leaseholder's clock - always leads the written_timestamp of any value in its keyspace written by a + always leads the local_timestamp of any value in its keyspace written by a prior leaseholder on its range, which is an important property for the correctness of observed timestamps. Second, it ensures that the leaseholder immediately views itself as the leaseholder. Third, it ensures that if the @@ -97,9 +95,6 @@ TODO(nvanbenschoten): Update the above on written timestamps after #72121. this later lease's start time could be pulled from the local clock and be guaranteed to receive an even greater starting timestamp. - TODO(nvanbenschoten): the written_timestamp concept does not yet exist in - code. It will be introduced in the replacement to #72121. - - Range merges (Raft + BatchRequest channels). During a merge of two ranges, the right-hand side of the merge passes a "frozen timestamp" clock reading from the right-hand side leaseholder, through the merge transaction @@ -109,21 +104,21 @@ TODO(nvanbenschoten): Update the above on written timestamps after #72121. merge and officially takes control of the combined range, it forwards its HLC to this frozen timestamp. Like the previous interaction, this one is also necessary to ensure that the leaseholder of the joint range has a clock that - leads the written_timestamp of any value in its keyspace, even one written + leads the local_timestamp of any value in its keyspace, even one written originally on the right-hand side range. - Observed timestamps (Raft + BatchRequest channels). During the lifetime of a transaction, its coordinator issues BatchRequests to other nodes in the cluster. Each time a given transaction visits a node for the first time, it captures an observation from the node's HLC. Separately, when a leaseholder - on a given node serves a write, it ensures that the node's HLC clock is >= - the written_timestamp of the write. This written_timestamp is retained even - if an intent is moved to a higher timestamp if it is asynchronously resolved. - As a result, these "observed timestamps" captured during the lifetime of a - transaction can be used to make a claim about values that could not have been - written yet at the time that the transaction first visited the node, and by - extension, at the time that the transaction began. This allows the - transaction to avoid uncertainty restarts in some circumstances. + on a given node serves a write, it assigns the write a local_timestamp from + its node's HLC clock. This local_timestamp is retained even if an intent is + moved to a higher timestamp if it is asynchronously resolved. As a result, + these "observed timestamps" captured during the lifetime of a transaction can + be used to make a claim about values that could not have been written yet at + the time that the transaction first visited the node, and by extension, at + the time that the transaction began. This allows the transaction to avoid + uncertainty restarts in some circumstances. A variant of this same mechanism applies to non-transactional requests that defer their timestamp allocation to the leaseholder of their (single) range. From 1bcf95072ea6fb0ddcc6868e68a60a020ef16fd1 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Fri, 15 Apr 2022 14:00:13 -0400 Subject: [PATCH 6/9] kv/storage: add cluster version gates for local timestamp This commit adds a cluster version gate and a cluster setting for local timestamps, to assist with their migration into an existing cluster. This fixes mixed-version clusters' interaction with local timestamps. --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 18 + pkg/clusterversion/key_string.go | 6 +- pkg/cmd/roachtest/tests/acceptance.go | 1 - pkg/kv/kvserver/spanset/batch.go | 4 + pkg/storage/engine.go | 5 + pkg/storage/intent_reader_writer.go | 5 +- pkg/storage/intent_reader_writer_test.go | 2 +- pkg/storage/mvcc.go | 4 +- pkg/storage/mvcc_history_test.go | 13 +- pkg/storage/pebble.go | 49 +- pkg/storage/pebble_batch.go | 22 +- pkg/storage/sst_writer.go | 5 + ...uncertainty_limit_disable_local_timestamps | 2945 +++++++++++++++++ 15 files changed, 3059 insertions(+), 24 deletions(-) create mode 100644 pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index d27c9fa67e3b..9bcd02effe1e 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -282,4 +282,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 22.1 set the active cluster version in the format '.' +version version 22.1-4 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 2232bd9b3a14..3513772db725 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -213,6 +213,6 @@ trace.opentelemetry.collectorstringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabledbooleantrueif set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collectorstringthe address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -versionversion22.1set the active cluster version in the format '.' +versionversion22.1-4set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 23cf63acfb17..3edd5165471f 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -349,6 +349,14 @@ const ( // V22_1 is CockroachDB v22.1. It's used for all v22.1.x patch releases. V22_1 + // v22.2 versions. + // + // Start22_2 demarcates work towards CockroachDB v22.2. + Start22_2 + + // LocalTimestamps enables the use of local timestamps in MVCC values. + LocalTimestamps + // ************************************************* // Step (1): Add new versions here. // Do not add new versions to a patch release. @@ -604,6 +612,16 @@ var versionsSingleton = keyedVersions{ Version: roachpb.Version{Major: 22, Minor: 1}, }, + // v22.2 versions. Internal versions must be even. + { + Key: Start22_2, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 2}, + }, + { + Key: LocalTimestamps, + Version: roachpb.Version{Major: 22, Minor: 1, Internal: 4}, + }, + // ************************************************* // Step (2): Add new versions here. // Do not add new versions to a patch release. diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index c0a318499161..5fd983af0225 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -63,11 +63,13 @@ func _() { _ = x[PreSeedSpanCountTable-52] _ = x[SeedSpanCountTable-53] _ = x[V22_1-54] + _ = x[Start22_2-55] + _ = x[LocalTimestamps-56] } -const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1" +const _Key_name = "V21_2Start22_1TargetBytesAvoidExcessAvoidDrainingNamesDrainingNamesMigrationTraceIDDoesntImplyStructuredRecordingAlterSystemTableStatisticsAddAvgSizeColAlterSystemStmtDiagReqsMVCCAddSSTableInsertPublicSchemaNamespaceEntryOnRestoreUnsplitRangesInAsyncGCJobsValidateGrantOptionPebbleFormatBlockPropertyCollectorProbeRequestSelectRPCsTakeTracingInfoInbandPreSeedTenantSpanConfigsSeedTenantSpanConfigsPublicSchemasWithDescriptorsEnsureSpanConfigReconciliationEnsureSpanConfigSubscriptionEnableSpanConfigStoreScanWholeRowsSCRAMAuthenticationUnsafeLossOfQuorumRecoveryRangeLogAlterSystemProtectedTimestampAddColumnEnableProtectedTimestampsForTenantDeleteCommentsWithDroppedIndexesRemoveIncompatibleDatabasePrivilegesAddRaftAppliedIndexTermMigrationPostAddRaftAppliedIndexTermMigrationDontProposeWriteTimestampForLeaseTransfersTenantSettingsTableEnablePebbleFormatVersionBlockPropertiesDisableSystemConfigGossipTriggerMVCCIndexBackfillerEnableLeaseHolderRemovalBackupResolutionInJobLooselyCoupledRaftLogTruncationChangefeedIdlenessBackupDoesNotOverwriteLatestAndCheckpointEnableDeclarativeSchemaChangerRowLevelTTLPebbleFormatSplitUserKeysMarkedIncrementalBackupSubdirDateStyleIntervalStyleCastRewriteEnableNewStoreRebalancerClusterLocksVirtualTableAutoStatsTableSettingsForecastStatsSuperRegionsEnableNewChangefeedOptionsSpanCountTablePreSeedSpanCountTableSeedSpanCountTableV22_1Start22_2LocalTimestamps" -var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 455, 483, 504, 517, 536, 570, 608, 642, 674, 710, 742, 778, 820, 839, 879, 911, 930, 954, 975, 1006, 1024, 1065, 1095, 1106, 1137, 1160, 1193, 1217, 1241, 1263, 1276, 1288, 1314, 1328, 1349, 1367, 1372} +var _Key_index = [...]uint16{0, 5, 14, 36, 54, 76, 113, 152, 175, 189, 230, 256, 275, 309, 321, 352, 376, 397, 425, 455, 483, 504, 517, 536, 570, 608, 642, 674, 710, 742, 778, 820, 839, 879, 911, 930, 954, 975, 1006, 1024, 1065, 1095, 1106, 1137, 1160, 1193, 1217, 1241, 1263, 1276, 1288, 1314, 1328, 1349, 1367, 1372, 1381, 1396} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/cmd/roachtest/tests/acceptance.go b/pkg/cmd/roachtest/tests/acceptance.go index c0c45fb1efb1..d489c182be63 100644 --- a/pkg/cmd/roachtest/tests/acceptance.go +++ b/pkg/cmd/roachtest/tests/acceptance.go @@ -52,7 +52,6 @@ func registerAcceptance(r registry.Registry) { }, { name: "version-upgrade", - skip: "WIP: unskip when version checks are added to local_timestamp writes", fn: func(ctx context.Context, t test.Test, c cluster.Cluster) { runVersionUpgrade(ctx, t, c) }, diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index a8ac49d22083..17e095ea3ed2 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -662,6 +662,10 @@ func (s spanSetWriter) LogLogicalOp( s.w.LogLogicalOp(op, details) } +func (s spanSetWriter) ShouldWriteLocalTimestamps(ctx context.Context) bool { + return s.w.ShouldWriteLocalTimestamps(ctx) +} + // ReadWriter is used outside of the spanset package internally, in ccl. type ReadWriter struct { spanSetReader diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index fa52de4a8f8d..1370c4245a0f 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -645,6 +645,11 @@ type Writer interface { // // It is safe to modify the contents of the arguments after it returns. SingleClearEngineKey(key EngineKey) error + + // ShouldWriteLocalTimestamps is only for internal use in the storage package. + // This method is temporary, to handle the transition from clusters where not + // all nodes understand local timestamps. + ShouldWriteLocalTimestamps(ctx context.Context) bool } // ReadWriter is the read/write interface to an engine's data. diff --git a/pkg/storage/intent_reader_writer.go b/pkg/storage/intent_reader_writer.go index 3217153b8950..15b55297f4e2 100644 --- a/pkg/storage/intent_reader_writer.go +++ b/pkg/storage/intent_reader_writer.go @@ -30,9 +30,8 @@ type intentDemuxWriter struct { w Writer } -func wrapIntentWriter(ctx context.Context, w Writer) intentDemuxWriter { - idw := intentDemuxWriter{w: w} - return idw +func wrapIntentWriter(w Writer) intentDemuxWriter { + return intentDemuxWriter{w: w} } // ClearIntent has the same behavior as Writer.ClearIntent. buf is used as diff --git a/pkg/storage/intent_reader_writer_test.go b/pkg/storage/intent_reader_writer_test.go index baab535d195a..55bceaedfe2d 100644 --- a/pkg/storage/intent_reader_writer_test.go +++ b/pkg/storage/intent_reader_writer_test.go @@ -200,7 +200,7 @@ func TestIntentDemuxWriter(t *testing.T) { // This is a low-level test that explicitly wraps the writer, so it // doesn't matter how the original call to createTestPebbleEngine // behaved in terms of separated intents config. - w = wrapIntentWriter(context.Background(), &pw) + w = wrapIntentWriter(&pw) return "" case "put-intent": pw.reset() diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 252a14f9805a..3ee16397361c 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -1595,7 +1595,7 @@ func mvccPutInternal( versionValue := MVCCValue{} versionValue.Value = value versionValue.LocalTimestamp = localTimestamp - if !versionValue.LocalTimestampNeeded(versionKey) { + if !versionValue.LocalTimestampNeeded(versionKey) || !writer.ShouldWriteLocalTimestamps(ctx) { versionValue.LocalTimestamp = hlc.ClockTimestamp{} } @@ -3128,7 +3128,7 @@ func mvccResolveWriteIntent( newValue := oldValue newValue.LocalTimestamp = oldValue.GetLocalTimestamp(oldKey) newValue.LocalTimestamp.Forward(intent.ClockWhilePending.Timestamp) - if !newValue.LocalTimestampNeeded(newKey) { + if !newValue.LocalTimestampNeeded(newKey) || !rw.ShouldWriteLocalTimestamps(ctx) { newValue.LocalTimestamp = hlc.ClockTimestamp{} } diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 5b5e71ae7fa9..5834e73d2503 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "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/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -99,18 +98,16 @@ func TestMVCCHistories(t *testing.T) { datadriven.Walk(t, testutils.TestDataPath(t, "mvcc_histories"), func(t *testing.T, path string) { // We start from a clean slate in every test file. - engine, err := Open(ctx, InMemory(), CacheSize(1<<20 /* 1 MiB */), - func(cfg *engineConfig) error { - // Latest cluster version, since these tests are not ones where we - // are examining differences related to separated intents. - cfg.Settings = cluster.MakeTestingClusterSettings() - return nil - }) + engine, err := Open(ctx, InMemory(), CacheSize(1<<20 /* 1 MiB */)) if err != nil { t.Fatal(err) } defer engine.Close() + if strings.Contains(path, "_disable_local_timestamps") { + localTimestampsEnabled.Override(ctx, &engine.settings.SV, false) + } + reportDataEntries := func(buf *redact.StringBuilder) error { hasData := false err := engine.MVCCIterate(span.Key, span.EndKey, MVCCKeyAndIntentsIterKind, func(r MVCCKeyValue) error { diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 6dbc0fec1e64..98cf66cb64d9 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -924,7 +924,7 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { p.makeMetricEtcEventListener(ctx), ) p.eventListener = &cfg.Opts.EventListener - p.wrappedIntentWriter = wrapIntentWriter(ctx, p) + p.wrappedIntentWriter = wrapIntentWriter(p) // Read the current store cluster version. storeClusterVersion, err := getMinVersion(unencryptedFS, cfg.Dir) @@ -1303,6 +1303,45 @@ func (p *Pebble) LogLogicalOp(op MVCCLogicalOpType, details MVCCLogicalOpDetails // No-op. Logical logging disabled. } +// localTimestampsEnabled controls whether local timestamps are written in MVCC +// values. A true setting is also gated on clusterversion.LocalTimestamps. After +// all nodes in a cluster are at or beyond clusterversion.LocalTimestamps, +// different nodes will see the version state transition at different times. +// Nodes that have not yet seen the transition may remove the local timestamp +// from an intent that has one during intent resolution. This will not cause +// problems. +// +// TODO(nvanbenschoten): remove this cluster setting and its associated plumbing +// when removing the cluster version, once we're confident in the efficacy and +// stability of local timestamps. +var localTimestampsEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "storage.transaction.local_timestamps.enabled", + "if enabled, MVCC keys will be written with local timestamps", + true, +) + +func shouldWriteLocalTimestamps(ctx context.Context, settings *cluster.Settings) bool { + if !localTimestampsEnabled.Get(&settings.SV) { + // Not enabled. + return false + } + ver := settings.Version.ActiveVersionOrEmpty(ctx) + if ver == (clusterversion.ClusterVersion{}) { + // Some tests fail to configure settings. In these cases, assume that it + // is safe to write local timestamps. + return true + } + return ver.IsActive(clusterversion.LocalTimestamps) +} + +// ShouldWriteLocalTimestamps implements the Writer interface. +func (p *Pebble) ShouldWriteLocalTimestamps(ctx context.Context) bool { + // This is not fast. Pebble should not be used by writers that want + // performance. They should use pebbleBatch. + return shouldWriteLocalTimestamps(ctx, p.settings) +} + // Attrs implements the Engine interface. func (p *Pebble) Attrs() roachpb.Attributes { return p.attrs @@ -1539,7 +1578,7 @@ func (p *Pebble) GetAuxiliaryDir() string { // NewBatch implements the Engine interface. func (p *Pebble) NewBatch() Batch { - return newPebbleBatch(p.db, p.db.NewIndexedBatch(), false /* writeOnly */) + return newPebbleBatch(p.db, p.db.NewIndexedBatch(), false /* writeOnly */, p.settings) } // NewReadOnly implements the Engine interface. @@ -1549,7 +1588,7 @@ func (p *Pebble) NewReadOnly(durability DurabilityRequirement) ReadWriter { // NewUnindexedBatch implements the Engine interface. func (p *Pebble) NewUnindexedBatch(writeOnly bool) Batch { - return newPebbleBatch(p.db, p.db.NewBatch(), writeOnly) + return newPebbleBatch(p.db, p.db.NewBatch(), writeOnly, p.settings) } // NewSnapshot implements the Engine interface. @@ -2113,6 +2152,10 @@ func (p *pebbleReadOnly) LogLogicalOp(op MVCCLogicalOpType, details MVCCLogicalO panic("not implemented") } +func (p *pebbleReadOnly) ShouldWriteLocalTimestamps(ctx context.Context) bool { + panic("not implemented") +} + // pebbleSnapshot represents a snapshot created using Pebble.NewSnapshot(). type pebbleSnapshot struct { snapshot *pebble.Snapshot diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index 2ac471bc2188..b5450bc94257 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -16,6 +16,7 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -56,6 +57,10 @@ type pebbleBatch struct { wrappedIntentWriter intentDemuxWriter // scratch space for wrappedIntentWriter. scratch []byte + + settings *cluster.Settings + shouldWriteLocalTimestamps bool + shouldWriteLocalTimestampsCached bool } var _ Batch = &pebbleBatch{} @@ -67,7 +72,9 @@ var pebbleBatchPool = sync.Pool{ } // Instantiates a new pebbleBatch. -func newPebbleBatch(db *pebble.DB, batch *pebble.Batch, writeOnly bool) *pebbleBatch { +func newPebbleBatch( + db *pebble.DB, batch *pebble.Batch, writeOnly bool, settings *cluster.Settings, +) *pebbleBatch { pb := pebbleBatchPool.Get().(*pebbleBatch) *pb = pebbleBatch{ db: db, @@ -94,8 +101,9 @@ func newPebbleBatch(db *pebble.DB, batch *pebble.Batch, writeOnly bool) *pebbleB reusable: true, }, writeOnly: writeOnly, + settings: settings, } - pb.wrappedIntentWriter = wrapIntentWriter(context.Background(), pb) + pb.wrappedIntentWriter = wrapIntentWriter(pb) return pb } @@ -540,3 +548,13 @@ func (p *pebbleBatch) Repr() []byte { copy(reprCopy, repr) return reprCopy } + +// ShouldWriteLocalTimestamps implements the Writer interface. +func (p *pebbleBatch) ShouldWriteLocalTimestamps(ctx context.Context) bool { + // pebbleBatch is short-lived, so cache the value for performance. + if !p.shouldWriteLocalTimestampsCached { + p.shouldWriteLocalTimestamps = shouldWriteLocalTimestamps(ctx, p.settings) + p.shouldWriteLocalTimestampsCached = true + } + return p.shouldWriteLocalTimestamps +} diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index a7dac4766068..917fb551b4c7 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -334,6 +334,11 @@ func (fw *SSTWriter) Close() { fw.fw = nil } +// ShouldWriteLocalTimestamps implements the Writer interface. +func (fw *SSTWriter) ShouldWriteLocalTimestamps(context.Context) bool { + return false +} + // MemFile is a file-like struct that buffers all data written to it in memory. // Implements the writeCloseSyncer interface and is intended for use with // SSTWriter. diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps new file mode 100644 index 000000000000..e5def62954ea --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps @@ -0,0 +1,2945 @@ +# Setup: +# +# k1: value @ ts 20 +# k1: value @ ts 10 +# +# k2: value @ ts 20 +# k2: value @ ts 10, localTs 5 +# +# k3: value @ ts 20, localTs 10 +# k3: value @ ts 10 +# +# k4: value @ ts 20, localTs 10 +# k4: value @ ts 10, localTs 5 +# +# k5: intent @ ts 20 +# k5: value @ ts 10 +# +# k6: intent @ ts 20 +# k6: value @ ts 10, localTs 5 +# +# k7: intent @ ts 20, localTs 10 +# k7: value @ ts 10 +# +# k8: intent @ ts 20, localTs 10 +# k8: value @ ts 10, localTs 5 +# + +run ok +with k=k1 + put v=v1 ts=10,0 + put v=v2 ts=20,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 + +run ok +with k=k2 + put v=v3 ts=10,0 localTs=5,0 + put v=v4 ts=20,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 + +run ok +with k=k3 + put v=v5 ts=10,0 + put v=v6 ts=20,0 localTs=10,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 + +run ok +with k=k4 + put v=v7 ts=10,0 localTs=5,0 + put v=v8 ts=20,0 localTs=10,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0 -> /BYTES/v7 + +run ok +with k=k5 + put v=v9 ts=10,0 + txn_begin t=A ts=20,0 + put t=A v=v10 +---- +>> at end: +txn: "A" meta={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0 -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 + +run ok +with k=k6 + put v=v11 ts=10,0 localTs=5,0 + txn_begin t=B ts=20,0 + put t=B v=v12 +---- +>> at end: +txn: "B" meta={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0 -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0 -> /BYTES/v11 + +run ok +with k=k7 + put v=v13 ts=10,0 + txn_begin t=C ts=20,0 + put t=C v=v14 localTs=10,0 +---- +>> at end: +txn: "C" meta={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0 -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0 -> /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0 -> /BYTES/v14 +data: "k7"/10.000000000,0 -> /BYTES/v13 + +run ok +with k=k8 + put v=v15 ts=10,0 localTs=5,0 + txn_begin t=D ts=20,0 + put t=D v=v16 localTs=10,0 +---- +>> at end: +txn: "D" meta={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0 -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0 -> /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0 -> /BYTES/v14 +data: "k7"/10.000000000,0 -> /BYTES/v13 +meta: "k8"/0,0 -> txn={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k8"/20.000000000,0 -> /BYTES/v16 +data: "k8"/10.000000000,0 -> /BYTES/v15 + +# Test cases: +# +# for ts in (5, 10, 15, 20, 25): +# for localUncertaintyLimit in (5, 10, 15, 20, 25): +# if localUncertaintyLimit < ts: continue +# for globalUncertaintyLimit in (5, 10, 15, 20, 25): +# if globalUncertaintyLimit < ts: continue +# if globalUncertaintyLimit <= localUncertaintyLimit: continue +# for k in (k1, k2, k3, k4, k5, k6, k7, k8): +# for op in (get, scan): +# testCase() +# + +run ok +txn_begin t=txn1 ts=5,0 globalUncertaintyLimit=10,0 +---- +>> at end: +txn: "txn1" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=10.000000000,0 + +run ok +get t=txn1 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn1 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get t=txn1 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan t=txn1 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get t=txn1 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn1 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get t=txn1 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan t=txn1 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get t=txn1 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn1 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get t=txn1 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan t=txn1 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get t=txn1 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn1 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get t=txn1 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan t=txn1 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + + +run ok +txn_begin t=txn2 ts=5,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn2" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=15.000000000,0 + +run ok +get t=txn2 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn2 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get t=txn2 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan t=txn2 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get t=txn2 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn2 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get t=txn2 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan t=txn2 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get t=txn2 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn2 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get t=txn2 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan t=txn2 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get t=txn2 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn2 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get t=txn2 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan t=txn2 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + + +run ok +txn_begin t=txn3 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn3" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn3 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn3 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get t=txn3 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan t=txn3 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get t=txn3 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn3 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get t=txn3 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan t=txn3 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get t=txn3 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn3 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get t=txn3 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan t=txn3 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get t=txn3 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn3 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get t=txn3 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan t=txn3 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + + +run ok +txn_begin t=txn4 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn4" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn4 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn4 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get t=txn4 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan t=txn4 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get t=txn4 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn4 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get t=txn4 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan t=txn4 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get t=txn4 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn4 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get t=txn4 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan t=txn4 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get t=txn4 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn4 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get t=txn4 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan t=txn4 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + + +run ok +txn_begin t=txn5 ts=5,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn5" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=15.000000000,0 + +run error +get t=txn5 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn6 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn6" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run error +get t=txn6 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn7 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn7" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn7 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn8 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn8" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run error +get t=txn8 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn9 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn9" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn9 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn10 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn10" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn10 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn11 ts=10,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn11" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=15.000000000,0 + +run ok +get t=txn11 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn11 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn11 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn11 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn11 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn11 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn11 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn11 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn11 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn11 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn11 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn11 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn11 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn11 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn11 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn11 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn12 ts=10,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn12" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn12 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn12 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn12 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn12 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn12 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn12 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn12 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn12 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn12 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn12 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn12 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn12 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn12 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn12 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn12 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn12 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn13 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn13" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn13 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn13 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn13 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn13 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn13 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn13 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn13 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn13 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn13 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn13 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn13 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn13 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn13 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn13 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn13 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn13 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn14 ts=10,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn14" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn14 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn14 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn14 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn14 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn14 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn14 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn14 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn14 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn14 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn14 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn14 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn14 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn14 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn14 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn14 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn14 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn15 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn15" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn15 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn15 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn15 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn15 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn15 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn15 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn15 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn15 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn15 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn15 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn15 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn15 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn15 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn15 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn15 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn15 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn16 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn16" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn16 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn17 ts=15,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn17" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn17 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn17 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn17 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn17 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn17 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn17 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn17 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn17 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn17 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn17 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn17 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn17 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn17 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn17 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn17 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn17 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn18 ts=15,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn18" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn18 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn18 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn18 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn18 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn18 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn18 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn18 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn18 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn18 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn18 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn18 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn18 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn18 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn18 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn18 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn18 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn19 ts=15,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn19" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn19 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn20 ts=20,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn20" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn20 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +scan t=txn20 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +get t=txn20 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +scan t=txn20 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +get t=txn20 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> /BYTES/v6 @20.000000000,0 + +run ok +scan t=txn20 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3" -> /BYTES/v6 @20.000000000,0 + +run ok +get t=txn20 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> /BYTES/v8 @20.000000000,0 + +run ok +scan t=txn20 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4" -> /BYTES/v8 @20.000000000,0 + +run error +get t=txn20 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +scan t=txn20 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +get t=txn20 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +scan t=txn20 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +get t=txn20 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +scan t=txn20 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +get t=txn20 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +run error +scan t=txn20 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +# A subset of the previous test cases, but with non-transactional reads: +# +# for ts in (5, 15, 25): +# for localUncertaintyLimit in (5, 15, 25): +# if localUncertaintyLimit < ts: continue +# for globalUncertaintyLimit in (5, 15, 25): +# if globalUncertaintyLimit < ts: continue +# if globalUncertaintyLimit < localUncertaintyLimit: continue +# for k in (k1, k2, k3, k4, k5, k6, k7, k8): +# for op in (get, scan): +# testCase() +# + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> + +run ok +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> + +run ok +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> + +run ok +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> + +run ok +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> + +run ok +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> + +run ok +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> + +run ok +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> + +run ok +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> + +run ok +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> + +run ok +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> + +run ok +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> + +run ok +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> + +run error +get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + +run error +get k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +scan k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +get k=k2 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +scan k=k2 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +get k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> /BYTES/v6 @20.000000000,0 + +run ok +scan k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3" -> /BYTES/v6 @20.000000000,0 + +run ok +get k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> /BYTES/v8 @20.000000000,0 + +run ok +scan k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4" -> /BYTES/v8 @20.000000000,0 + +run error +get k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +scan k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +get k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +scan k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +get k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +scan k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +get k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +run error +scan k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" From a3e608d5abc8e2b3a364dc4736c0ed83ce88a065 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Sun, 8 May 2022 00:22:40 -0400 Subject: [PATCH 7/9] storage: store encoded MVCCValues in SequencedIntents This commit switches from storing encoded `roachpb.Value`s to storing encoded `storage.MVCCValue`s in `MVCCMetadata`'s `SequencedIntent`s. Doing so ensures that MVCCValue headers are not lost when an intent is rolled back. This is important to avoid losing the local timestamp of values in a key's intent history. Failure to do so could allow for stale reads. --- pkg/roachpb/data.go | 1 - pkg/storage/enginepb/mvcc.proto | 2 +- pkg/storage/mvcc.go | 112 +++++++++--------- pkg/storage/mvcc_value.go | 12 ++ pkg/storage/pebble_mvcc_scanner.go | 8 +- .../ignored_seq_nums_with_local_timestamps | 46 +++++++ .../testdata/mvcc_histories/local_timestamp | 32 ++--- 7 files changed, 138 insertions(+), 75 deletions(-) create mode 100644 pkg/storage/testdata/mvcc_histories/ignored_seq_nums_with_local_timestamps diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index e6814bd9b86d..9efc6fc14f8d 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -2489,7 +2489,6 @@ var _ = (SequencedWriteBySeq{}).Find func init() { // Inject the format dependency into the enginepb package. enginepb.FormatBytesAsKey = func(k []byte) string { return Key(k).String() } - enginepb.FormatBytesAsValue = func(v []byte) string { return Value{RawBytes: v}.PrettyPrint() } } // SafeValue implements the redact.SafeValue interface. diff --git a/pkg/storage/enginepb/mvcc.proto b/pkg/storage/enginepb/mvcc.proto index 6faa0c8bf355..93bda348d0af 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -58,7 +58,7 @@ message MVCCMetadata { // the IntentHistory. optional int32 sequence = 1 [(gogoproto.nullable) = false, (gogoproto.casttype) = "TxnSeq"]; // Value is the value written to the key as part of the transaction at - // the above Sequence. Value uses the roachpb.Value encoding. + // the above Sequence. Value uses the storage.MVCCValue encoding. optional bytes value = 2; } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 3ee16397361c..9f6f5bc3cabe 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -1171,24 +1171,27 @@ func replayTransactionalWrite( txn *roachpb.Transaction, valueFn func(optionalValue) (roachpb.Value, error), ) error { - var found bool - var writtenValue []byte + var writtenValue optionalValue var err error if txn.Sequence == meta.Txn.Sequence { // This is a special case. This is when the intent hasn't made it // to the intent history yet. We must now assert the value written // in the intent to the value we're trying to write. - exVal, _, err := mvccGet(ctx, iter, key, timestamp, MVCCGetOptions{Txn: txn, Tombstones: true}) + writtenValue, _, err = mvccGet(ctx, iter, key, timestamp, MVCCGetOptions{Txn: txn, Tombstones: true}) if err != nil { return err } - writtenValue = exVal.RawBytes - found = true } else { // Get the value from the intent history. - writtenValue, found = meta.GetIntentValue(txn.Sequence) + if intentValRaw, ok := meta.GetIntentValue(txn.Sequence); ok { + intentVal, err := DecodeMVCCValue(intentValRaw) + if err != nil { + return err + } + writtenValue = makeOptionalValue(intentVal.Value) + } } - if !found { + if !writtenValue.exists { // NB: This error may be due to a batched `DelRange` operation that, upon being replayed, finds a new key to delete. // See issue #71236 for more explanation. err := errors.AssertionFailedf("transaction %s with sequence %d missing an intent with lower sequence %d", @@ -1215,9 +1218,11 @@ func replayTransactionalWrite( // If the previous value was found in the IntentHistory, // simply apply the value function to the historic value // to get the would-be value. - prevVal := prevIntent.Value - - exVal = makeOptionalValue(roachpb.Value{RawBytes: prevVal}) + prevIntentVal, err := DecodeMVCCValue(prevIntent.Value) + if err != nil { + return err + } + exVal = makeOptionalValue(prevIntentVal.Value) } else { // If the previous value at the key wasn't written by this // transaction, or it was hidden by a rolled back seqnum, we look at @@ -1239,9 +1244,9 @@ func replayTransactionalWrite( // To ensure the transaction is idempotent, we must assert that the // calculated value on this replay is the same as the one we've previously // written. - if !bytes.Equal(value.RawBytes, writtenValue) { + if !bytes.Equal(value.RawBytes, writtenValue.RawBytes) { return errors.AssertionFailedf("transaction %s with sequence %d has a different value %+v after recomputing from what was written: %+v", - txn.ID, txn.Sequence, value.RawBytes, writtenValue) + txn.ID, txn.Sequence, value.RawBytes, writtenValue.RawBytes) } return nil } @@ -1402,6 +1407,8 @@ func mvccPutInternal( // We're overwriting the intent that was present at this key, before we do // that though - we must record the older value in the IntentHistory. + oldVersionKey := metaKey + oldVersionKey.Timestamp = metaTimestamp // But where to find the older value? There are 4 cases: // - last write inside txn, same epoch, seqnum of last write is not @@ -1422,23 +1429,36 @@ func mvccPutInternal( // rolled back, either due to transaction retries or transaction savepoint // rollbacks.) var exVal optionalValue - // Set to true when the current provisional value is not ignored due to - // a txn restart or a savepoint rollback. - var curProvNotIgnored bool + // Set when the current provisional value is not ignored due to a txn + // restart or a savepoint rollback. Represents an encoded MVCCValue. + var curProvValRaw []byte if txn.Epoch == meta.Txn.Epoch /* last write inside txn */ { if !enginepb.TxnSeqIsIgnored(meta.Txn.Sequence, txn.IgnoredSeqNums) { - // Seqnum of last write is not ignored. Retrieve the value - // using a consistent read. - exVal, _, err = mvccGet(ctx, iter, key, readTimestamp, MVCCGetOptions{Txn: txn, Tombstones: true}) + // Seqnum of last write is not ignored. Retrieve the value. + iter.SeekGE(oldVersionKey) + if valid, err := iter.Valid(); err != nil { + return err + } else if !valid && !iter.UnsafeKey().Equal(oldVersionKey) { + return errors.Errorf("existing intent value missing: %s", oldVersionKey) + } + + // NOTE: we use Value instead of UnsafeValue so that we can move the + // iterator below without invalidating this byte slice. + curProvValRaw = iter.Value() + curIntentVal, err := DecodeMVCCValue(curProvValRaw) if err != nil { return err } - curProvNotIgnored = true + exVal = makeOptionalValue(curIntentVal.Value) } else { // Seqnum of last write was ignored. Try retrieving the value from the history. - prevIntent, prevValueWritten := meta.GetPrevIntentSeq(txn.Sequence, txn.IgnoredSeqNums) - if prevValueWritten { - exVal = makeOptionalValue(roachpb.Value{RawBytes: prevIntent.Value}) + prevIntent, prevIntentOk := meta.GetPrevIntentSeq(txn.Sequence, txn.IgnoredSeqNums) + if prevIntentOk { + prevIntentVal, err := DecodeMVCCValue(prevIntent.Value) + if err != nil { + return err + } + exVal = makeOptionalValue(prevIntentVal.Value) } } } @@ -1472,33 +1492,33 @@ func mvccPutInternal( // delete the old intent, taking care with MVCC stats. logicalOp = MVCCUpdateIntentOpType if metaTimestamp.Less(writeTimestamp) { - versionKey := metaKey - versionKey.Timestamp = metaTimestamp - { // If the older write intent has a version underneath it, we need to // read its size because its GCBytesAge contribution may change as we // move the intent above it. A similar phenomenon occurs in // MVCCResolveWriteIntent. - _, prevUnsafeVal, haveNextVersion, err := unsafeNextVersion(iter, versionKey) - if err != nil { + prevKey := oldVersionKey.Next() + iter.SeekGE(prevKey) + if valid, err := iter.Valid(); err != nil { return err - } - if haveNextVersion { - prevVal, ok, err := tryDecodeSimpleMVCCValue(prevUnsafeVal) - if !ok && err == nil { - prevVal, err = decodeExtendedMVCCValue(prevUnsafeVal) + } else if valid && iter.UnsafeKey().Key.Equal(prevKey.Key) { + prevUnsafeKey := iter.UnsafeKey() + if !prevUnsafeKey.IsValue() { + return errors.Errorf("expected an MVCC value key: %s", prevUnsafeKey) } + + prevValRaw := iter.UnsafeValue() + prevVal, err := DecodeMVCCValue(prevValRaw) if err != nil { return err } prevIsValue = prevVal.Value.IsPresent() - prevValSize = int64(len(prevUnsafeVal)) + prevValSize = int64(len(prevValRaw)) } iter = nil // prevent accidental use below } - if err := writer.ClearMVCC(versionKey); err != nil { + if err := writer.ClearMVCC(oldVersionKey); err != nil { return err } } else if writeTimestamp.Less(metaTimestamp) { @@ -1529,10 +1549,10 @@ func mvccPutInternal( // history if the current sequence number is not ignored. There's no // reason to add past committed values or a value already in the intent // history back into it. - if curProvNotIgnored { - prevIntentValBytes := exVal.RawBytes + if curProvValRaw != nil { + prevIntentValRaw := curProvValRaw prevIntentSequence := meta.Txn.Sequence - buf.newMeta.AddToIntentHistory(prevIntentSequence, prevIntentValBytes) + buf.newMeta.AddToIntentHistory(prevIntentSequence, prevIntentValRaw) } } else { buf.newMeta.IntentHistory = nil @@ -2676,24 +2696,6 @@ func MVCCResolveWriteIntent( return ok, err } -// unsafeNextVersion positions the iterator at the successor to latestKey. If this value -// exists and is a version of the same key, returns the UnsafeKey() and UnsafeValue() of that -// key-value pair along with `true`. -func unsafeNextVersion(iter MVCCIterator, latestKey MVCCKey) (MVCCKey, []byte, bool, error) { - // Compute the next possible mvcc value for this key. - nextKey := latestKey.Next() - iter.SeekGE(nextKey) - - if ok, err := iter.Valid(); err != nil || !ok || !iter.UnsafeKey().Key.Equal(latestKey.Key) { - return MVCCKey{}, nil, false /* never ok */, err - } - unsafeKey := iter.UnsafeKey() - if !unsafeKey.IsValue() { - return MVCCKey{}, nil, false, errors.Errorf("expected an MVCC value key: %s", unsafeKey) - } - return unsafeKey, iter.UnsafeValue(), true, nil -} - // iterForKeyVersions provides a subset of the functionality of MVCCIterator. // The expected use-case is when the iter is already positioned at the intent // (if one exists) for a particular key, or some version, and positioning diff --git a/pkg/storage/mvcc_value.go b/pkg/storage/mvcc_value.go index 86ea768eb0b6..587a42f0973f 100644 --- a/pkg/storage/mvcc_value.go +++ b/pkg/storage/mvcc_value.go @@ -12,6 +12,7 @@ package storage import ( "encoding/binary" + "fmt" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -243,3 +244,14 @@ func decodeExtendedMVCCValue(buf []byte) (MVCCValue, error) { v.Value.RawBytes = buf[headerSize:] return v, nil } + +func init() { + // Inject the format dependency into the enginepb package. + enginepb.FormatBytesAsValue = func(v []byte) string { + val, err := DecodeMVCCValue(v) + if err != nil { + return fmt.Sprintf("err=%v", err) + } + return val.String() + } +} diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 94c2d9e1a959..6699a940517d 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -739,7 +739,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // numbers) that we should read. If there exists a value in the intent // history that has a sequence number equal to or less than the read // sequence, read that value. - if value, found := p.getFromIntentHistory(); found { + if intentValueRaw, found := p.getFromIntentHistory(); found { // If we're adding a value due to a previous intent, we want to populate // the timestamp as of current metaTimestamp. Note that this may be // controversial as this maybe be neither the write timestamp when this @@ -751,7 +751,11 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // addAndAdvance to take an MVCCKey explicitly. p.curUnsafeKey.Timestamp = metaTS p.keyBuf = EncodeMVCCKeyToBuf(p.keyBuf[:0], p.curUnsafeKey) - return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.keyBuf, value) + p.curUnsafeValue, p.err = DecodeMVCCValue(intentValueRaw) + if p.err != nil { + return false + } + return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.keyBuf, p.curUnsafeValue.Value.RawBytes) } // 13. If no value in the intent history has a sequence number equal to // or less than the read, we must ignore the intents laid down by the diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_with_local_timestamps b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_with_local_timestamps new file mode 100644 index 000000000000..4b6f86792cf5 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_with_local_timestamps @@ -0,0 +1,46 @@ +# Perform some writes at various sequence numbers with local timestamps. + +run ok +with t=A + txn_begin ts=50 + txn_step seq=15 + put k=k v=a localTs=15,0 + txn_step seq=20 + put k=k v=b localTs=20,0 + txn_step seq=25 + put k=k v=c localTs=25,0 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=25} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=25} ts=50.000000000,0 del=false klen=12 vlen=19 ih={{15 vheader{ localTs=15.000000000,0 } /BYTES/a}{20 vheader{ localTs=20.000000000,0 } /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false +data: "k"/50.000000000,0 -> vheader{ localTs=25.000000000,0 } /BYTES/c + +# Rollback to a previous sequence number. Should be able to read before and +# after resolving the intent. + +run ok +with t=A + txn_ignore_seqs seqs=(25-25) + get k=k + resolve_intent k=k status=PENDING + get k=k +---- +get: "k" -> /BYTES/b @50.000000000,0 +get: "k" -> /BYTES/b @50.000000000,0 +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=25} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=19 ih={{15 vheader{ localTs=15.000000000,0 } /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +data: "k"/50.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/b + +# Rollback and commit at a previous sequence number. Committed value should have +# original local timestamp. This is important to avoid losing the local timestamp +# of values in a key's intent history. + +run ok +with t=A + txn_ignore_seqs seqs=(20-20) + resolve_intent k=k status=COMMITTED +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=25} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 +data: "k"/50.000000000,0 -> vheader{ localTs=15.000000000,0 } /BYTES/a diff --git a/pkg/storage/testdata/mvcc_histories/local_timestamp b/pkg/storage/testdata/mvcc_histories/local_timestamp index 1f2670f66d1e..79e71ec3ca9d 100644 --- a/pkg/storage/testdata/mvcc_histories/local_timestamp +++ b/pkg/storage/testdata/mvcc_histories/local_timestamp @@ -267,29 +267,29 @@ with t=A localTs=20 put k=k12 v=v2 ---- >> at end: -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k1"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k10"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k11"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k11"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k11"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k12"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k12"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k12"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k2"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k4"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k4"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k4"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k5"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k5"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k5"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k6"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k6"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k6"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k7"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k7"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k7"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k8"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k8"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k8"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k9"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k9"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k9"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 run ok @@ -313,12 +313,12 @@ txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0 data: "k10"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 data: "k11"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } /BYTES/v2 data: "k12"/40.000000000,0 -> /BYTES/v2 -meta: "k5"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k5"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k5"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k6"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k6"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k6"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 -meta: "k7"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k7"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k7"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } /BYTES/v2 -meta: "k8"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +meta: "k8"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=7 ih={{0 vheader{ localTs=10.000000000,0 } /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false data: "k8"/40.000000000,0 -> /BYTES/v2 data: "k9"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 From 52de4f9cf494651c3aec51a572d5d2f912e01fd4 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Sun, 8 May 2022 00:34:33 -0400 Subject: [PATCH 8/9] kv/uncertainty: assert that value and local timestamp are non-zero This commit adds an assertion to `Interval.IsUncertain` that the provided value and local timestamps are non-zero. --- pkg/kv/kvserver/uncertainty/interval.go | 6 ++++++ pkg/kv/kvserver/uncertainty/interval_test.go | 17 ----------------- 2 files changed, 6 insertions(+), 17 deletions(-) diff --git a/pkg/kv/kvserver/uncertainty/interval.go b/pkg/kv/kvserver/uncertainty/interval.go index 22c4e5520f78..cd427fda5e8d 100644 --- a/pkg/kv/kvserver/uncertainty/interval.go +++ b/pkg/kv/kvserver/uncertainty/interval.go @@ -52,6 +52,12 @@ type Interval struct { // timestamps is uncertain to a reader with a ReadTimestamp below the value's // version timestamp and with the specified uncertainty interval. func (in Interval) IsUncertain(valueTs hlc.Timestamp, localTs hlc.ClockTimestamp) bool { + if valueTs.IsEmpty() { + panic("unexpected empty value timestamp") + } + if localTs.IsEmpty() { + panic("unexpected empty local timestamp") + } if !in.LocalLimit.IsEmpty() && in.LocalLimit.Less(localTs) { // The reader has an observed timestamp that precedes the local timestamp of // this value. There is no uncertainty as the reader transaction must have diff --git a/pkg/kv/kvserver/uncertainty/interval_test.go b/pkg/kv/kvserver/uncertainty/interval_test.go index 065e2789522e..cd733e0be1a6 100644 --- a/pkg/kv/kvserver/uncertainty/interval_test.go +++ b/pkg/kv/kvserver/uncertainty/interval_test.go @@ -30,22 +30,6 @@ func TestInterval_IsUncertain(t *testing.T) { localLim, globalLim, valueTs, localTs hlc.Timestamp exp bool }{ - // Without local timestamp. localLim is ignored. - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(5), localTs: emptyTs, exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(10), localTs: emptyTs, exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), localTs: emptyTs, exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: emptyTs, exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: emptyTs, exp: false}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(5), localTs: emptyTs, exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(10), localTs: emptyTs, exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), localTs: emptyTs, exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: emptyTs, exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: emptyTs, exp: false}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(5), localTs: emptyTs, exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(10), localTs: emptyTs, exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), localTs: emptyTs, exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: emptyTs, exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: emptyTs, exp: false}, // With local timestamp equal to value timestamp. {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(5), localTs: makeTs(5), exp: true}, {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(10), exp: true}, @@ -94,7 +78,6 @@ func TestInterval_IsUncertain(t *testing.T) { {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(15), exp: false}, {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(20), exp: false}, // Empty uncertainty intervals. - {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(10), localTs: emptyTs, exp: false}, {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(10), localTs: makeTs(10), exp: false}, {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(10), localTs: makeTs(5), exp: false}, } From 720c8cf07fcf16edd9457a96027a9604dc6c121d Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Sun, 8 May 2022 23:09:59 -0400 Subject: [PATCH 9/9] storage: disable simple MVCC value encoding scheme under metamorphic testing This commit adds a metamorphic knob that randomly disables the simple MVCC value encoding scheme. Doing so ensures that code which interacts with encoded MVCC values does not mistake these values for encoded roachpb values. This could take place in two different ways: 1. broken code could assign an encoded MVCC value directly to a roachpb.Value's `RawBytes` field. This typically caused the test to fail with an error. 2. broken code could assume that a non-zero-length value was not a tombstone. This caused tests to fail in more obscure ways. The commit then fixes broken tests in one of three ways: - it fixes incorrect assumptions about the MVCC value encoding being equivalent to the roachpb value encoding. - it updates a few important tests (mostly related to MVCC stats) to work with and without the simple encoding. - it skips a few unimportant tests when the simple encoding scheme is disabled. --- .../backupccl/restore_data_processor_test.go | 6 +- .../batcheval/cmd_add_sstable_test.go | 12 +- pkg/kv/kvserver/batcheval/cmd_export_test.go | 1 + pkg/kv/kvserver/consistency_queue_test.go | 3 + pkg/kv/kvserver/gc/gc_iterator_test.go | 1 + pkg/kv/kvserver/mvcc_gc_queue_test.go | 2 + pkg/kv/kvserver/replica_test.go | 1 + pkg/sql/row/fetcher_mvcc_test.go | 12 +- pkg/storage/BUILD.bazel | 1 + pkg/storage/mvcc_history_test.go | 1 + pkg/storage/mvcc_incremental_iterator_test.go | 7 + pkg/storage/mvcc_stats_test.go | 253 +++++++++++++----- pkg/storage/mvcc_value.go | 36 ++- pkg/storage/mvcc_value_test.go | 1 + pkg/storage/pebble_test.go | 1 + pkg/storage/sst.go | 34 ++- pkg/storage/sst_iterator.go | 18 +- 17 files changed, 298 insertions(+), 92 deletions(-) diff --git a/pkg/ccl/backupccl/restore_data_processor_test.go b/pkg/ccl/backupccl/restore_data_processor_test.go index 1c929cb5bc60..d120c6a4e0d2 100644 --- a/pkg/ccl/backupccl/restore_data_processor_test.go +++ b/pkg/ccl/backupccl/restore_data_processor_test.go @@ -113,7 +113,11 @@ func slurpSSTablesLatestKey( } else if !ok || !it.UnsafeKey().Less(end) { break } - kvs = append(kvs, storage.MVCCKeyValue{Key: it.Key(), Value: it.Value()}) + val, err := storage.DecodeMVCCValue(it.Value()) + if err != nil { + t.Fatal(err) + } + kvs = append(kvs, storage.MVCCKeyValue{Key: it.Key(), Value: val.Value.RawBytes}) } return kvs } diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index b29e721d04bf..cf016db1222e 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -772,8 +772,10 @@ func TestEvalAddSSTable(t *testing.T) { ts := iter.Key().Timestamp.WallTime var value []byte if iter.Key().IsValue() { - if len(iter.Value()) > 0 { - value, err = roachpb.Value{RawBytes: iter.Value()}.GetBytes() + mvccVal, err := storage.DecodeMVCCValue(iter.Value()) + require.NoError(t, err) + if !mvccVal.IsTombstone() { + value, err = mvccVal.Value.GetBytes() require.NoError(t, err) } } else { @@ -1091,6 +1093,7 @@ func runTestDBAddSSTable( func TestAddSSTableMVCCStats(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + storage.SkipIfSimpleValueEncodingDisabled(t) const max = 1 << 10 ctx := context.Background() @@ -1209,6 +1212,7 @@ func TestAddSSTableMVCCStats(t *testing.T) { func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + storage.SkipIfSimpleValueEncodingDisabled(t) ctx := context.Background() st := cluster.MakeTestingClusterSettings() @@ -1467,7 +1471,9 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsClosedTS(t *testing.T) require.NoError(t, err) require.Len(t, kvs, 1) require.Equal(t, storage.MVCCKey{Key: roachpb.Key("key"), Timestamp: writeTS}, kvs[0].Key) - v, err := roachpb.Value{RawBytes: kvs[0].Value}.GetBytes() + mvccVal, err := storage.DecodeMVCCValue(kvs[0].Value) + require.NoError(t, err) + v, err := mvccVal.Value.GetBytes() require.NoError(t, err) require.Equal(t, "sst", string(v)) } diff --git a/pkg/kv/kvserver/batcheval/cmd_export_test.go b/pkg/kv/kvserver/batcheval/cmd_export_test.go index 837ff893d3bd..de6fd2d11681 100644 --- a/pkg/kv/kvserver/batcheval/cmd_export_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_export_test.go @@ -674,6 +674,7 @@ func assertEqualKVs( func TestRandomKeyAndTimestampExport(t *testing.T) { defer leaktest.AfterTest(t)() + storage.SkipIfSimpleValueEncodingDisabled(t) ctx := context.Background() diff --git a/pkg/kv/kvserver/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index 951f071e7aee..279968db5b8b 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -234,6 +234,9 @@ func TestCheckConsistencyInconsistent(t *testing.T) { // good to make sure we're overly redacting said diff. defer log.TestingSetRedactable(true)() + // Test expects simple MVCC value encoding. + storage.SkipIfSimpleValueEncodingDisabled(t) + // Test uses sticky registry to have persistent pebble state that could // be analyzed for existence of snapshots and to verify snapshot content // after failures. diff --git a/pkg/kv/kvserver/gc/gc_iterator_test.go b/pkg/kv/kvserver/gc/gc_iterator_test.go index 8c148f30d82e..6137c6d237e3 100644 --- a/pkg/kv/kvserver/gc/gc_iterator_test.go +++ b/pkg/kv/kvserver/gc/gc_iterator_test.go @@ -28,6 +28,7 @@ import ( // engine and then validating the state of the iterator as it iterates that // data. func TestGCIterator(t *testing.T) { + storage.SkipIfSimpleValueEncodingDisabled(t) // dataItem represents a version in the storage engine and optionally a // corresponding transaction which will make the MVCCKeyValue an intent. type dataItem struct { diff --git a/pkg/kv/kvserver/mvcc_gc_queue_test.go b/pkg/kv/kvserver/mvcc_gc_queue_test.go index b44d3c031a52..95d0eba35631 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -356,6 +356,7 @@ func (cws *cachedWriteSimulator) shouldQueue( func TestMVCCGCQueueMakeGCScoreRealistic(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + storage.SkipIfSimpleValueEncodingDisabled(t) cws := newCachedWriteSimulator(t) @@ -462,6 +463,7 @@ func TestMVCCGCQueueMakeGCScoreRealistic(t *testing.T) { func TestMVCCGCQueueProcess(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + storage.SkipIfSimpleValueEncodingDisabled(t) ctx := context.Background() tc := testContext{} stopper := stop.NewStopper() diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 90acce579380..a622cffd67ae 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -6247,6 +6247,7 @@ func verifyRangeStats( func TestRangeStatsComputation(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + storage.SkipIfSimpleValueEncodingDisabled(t) ctx := context.Background() tc := testContext{} stopper := stop.NewStopper() diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index 48f6495eee9e..5d108dbd2771 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -57,10 +57,14 @@ func slurpUserDataKVs(t testing.TB, e storage.Engine) []roachpb.KeyValue { if !it.UnsafeKey().IsValue() { return errors.Errorf("found intent key %v", it.UnsafeKey()) } - kvs = append(kvs, roachpb.KeyValue{ - Key: it.Key().Key, - Value: roachpb.Value{RawBytes: it.Value(), Timestamp: it.UnsafeKey().Timestamp}, - }) + mvccValue, err := storage.DecodeMVCCValue(it.Value()) + if err != nil { + t.Fatal(err) + } + value := mvccValue.Value + value.Timestamp = it.UnsafeKey().Timestamp + kv := roachpb.KeyValue{Key: it.Key().Key, Value: value} + kvs = append(kvs, kv) } return nil }) diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index af04e1c3b7b6..8f54ee576591 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -57,6 +57,7 @@ go_library( "//pkg/settings/cluster", "//pkg/storage/enginepb", "//pkg/storage/fs", + "//pkg/testutils/skip", "//pkg/util", "//pkg/util/bufalloc", "//pkg/util/encoding", diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 5834e73d2503..7c13fb62d13e 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -90,6 +90,7 @@ import ( func TestMVCCHistories(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + SkipIfSimpleValueEncodingDisabled(t) ctx := context.Background() diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index 79c1363b89f9..0ced12facc5e 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -497,6 +497,7 @@ func assertEqualKVs( func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + SkipIfSimpleValueEncodingDisabled(t) ctx := context.Background() var ( @@ -634,6 +635,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + SkipIfSimpleValueEncodingDisabled(t) ctx := context.Background() var ( @@ -764,6 +766,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { func TestMVCCIncrementalIteratorInlinePolicy(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + SkipIfSimpleValueEncodingDisabled(t) ctx := context.Background() var ( @@ -858,6 +861,7 @@ func TestMVCCIncrementalIteratorInlinePolicy(t *testing.T) { func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + SkipIfSimpleValueEncodingDisabled(t) ctx := context.Background() var ( @@ -1038,6 +1042,7 @@ func expectIntent(t *testing.T, iter SimpleMVCCIterator, intent roachpb.Intent) func TestMVCCIncrementalIterator(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + SkipIfSimpleValueEncodingDisabled(t) ctx := context.Background() var ( @@ -1318,6 +1323,7 @@ func TestMVCCIncrementalIteratorIntentRewrittenConcurrently(t *testing.T) { func TestMVCCIncrementalIteratorIntentDeletion(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + SkipIfSimpleValueEncodingDisabled(t) txn := func(key roachpb.Key, ts hlc.Timestamp) *roachpb.Transaction { return &roachpb.Transaction{ @@ -1535,6 +1541,7 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { func TestMVCCIterateTimeBound(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + SkipIfSimpleValueEncodingDisabled(t) dir, cleanupFn := testutils.TempDir(t) defer cleanupFn() diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 2d45b747c525..57c98d66e61f 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -77,6 +77,11 @@ func assertEqLocal(t *testing.T, rw ReadWriter, debug string, ms, expMS *enginep assertEqImpl(t, rw, debug, false /* globalKeys */, ms, expMS) } +var emptyMVCCValueHeaderSize = func() int64 { + var h enginepb.MVCCValueHeader + return extendedPreludeSize + int64(h.Size()) +}() + // TestMVCCStatsDeleteCommitMovesTimestamp exercises the case in which a value // is written, later deleted via an intent and the deletion committed at an even // higher timestamp. This exercises subtleties related to the implicit push of @@ -105,13 +110,16 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { mKeySize := int64(mvccKey(key).EncodedSize()) // 2 vKeySize := MVCCVersionTimestampSize // 12 vValSize := int64(len(value.RawBytes)) // 10 + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize // 17 + } expMS := enginepb.MVCCStats{ - LiveBytes: mKeySize + vKeySize + vValSize, // 24 + LiveBytes: mKeySize + vKeySize + vValSize, // 24[+7] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 14 KeyCount: 1, - ValBytes: vValSize, // 10 + ValBytes: vValSize, // 10[+7] ValCount: 1, LastUpdateNanos: 1e9, } @@ -201,18 +209,21 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { mValSize += 2 vKeySize := MVCCVersionTimestampSize // 12 vValSize := int64(len(value.RawBytes)) // 10 + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize // 17 + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+10 = 68[+2] + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 68[+2][+7] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 2+12 =14 KeyCount: 1, - ValBytes: mValSize + vValSize, // (46[+2])+10 = 54[+2] + ValBytes: mValSize + vValSize, // (46[+2])+(10[+7]) = 54[+2][+7] ValCount: 1, IntentCount: 1, SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // 12+10 = 22 + IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] GCBytesAge: 0, } assertEq(t, engine, "after put", aggMS, &expMS) @@ -233,7 +244,7 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { // that it now uses the extended MVCCValue encoding. vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 - vValSize += vValHeaderSize // 23 + vValSize = int64(len(value.RawBytes)) + vValHeaderSize // 23 expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 4e9, @@ -290,19 +301,22 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { mValSize += 2 vKeySize := MVCCVersionTimestampSize // 12 vValSize := int64(len(value.RawBytes)) // 10 + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize // 17 + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+10 = 70[+2] + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 70[+2][+7] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 2+12 = 14 KeyCount: 1, - ValBytes: mValSize + vValSize, // (46[+2])+10 = 54[+2] + ValBytes: mValSize + vValSize, // (46[+2])+(10[+7]) = 54[+2][+7] ValCount: 1, IntentAge: 0, IntentCount: 1, SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // 12+10 = 22 + IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] } assertEq(t, engine, "after put", aggMS, &expMS) @@ -323,7 +337,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { // that it now uses the extended MVCCValue encoding. vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 - vValSize += vValHeaderSize // 23 + vValSize = int64(len(value.RawBytes)) + vValHeaderSize // 23 expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 4e9, @@ -404,19 +418,23 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { vValSize := int64(len(value.RawBytes)) require.EqualValues(t, vValSize, 10) + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize + require.EqualValues(t, vValSize, 17) + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, - LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(46[+2])+12+10 = 70[+2] + LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 70[+2][+7] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 2+12 = 14 KeyCount: 1, - ValBytes: mVal1Size + vValSize, // (46[+2])+10 = 56([+2]) + ValBytes: mVal1Size + vValSize, // (46[+2])+(10[+7]) = 56[+2][+7] ValCount: 1, IntentAge: 0, IntentCount: 1, SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // 12+10 = 22 + IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] } assertEq(t, engine, "after put", aggMS, &expMS) @@ -430,19 +448,30 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { // Annoyingly, the new meta value is actually a little larger thanks to the // sequence number. Also since there was a write previously on the same // transaction, the IntentHistory will add a few bytes to the metadata. + encValue, err := EncodeMVCCValue(MVCCValue{Value: value}) + require.NoError(t, err) m2ValSize := int64((&enginepb.MVCCMetadata{ Timestamp: ts2.ToLegacyTimestamp(), Txn: &txn.TxnMeta, IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: value.RawBytes}, + {Sequence: 0, Value: encValue}, }, }).Size()) - require.EqualValues(t, m2ValSize, 64) + expM2ValSize := 64 + if disableSimpleValueEncoding { + expM2ValSize = 71 + } + require.EqualValues(t, m2ValSize, expM2ValSize) if err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { t.Fatal(err) } + vVal2Size := int64(0) // tombstone + if disableSimpleValueEncoding { + vVal2Size = emptyMVCCValueHeaderSize // 7 + } + expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 2e9, LiveBytes: 0, @@ -453,12 +482,12 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { // One versioned key counts for vKeySize. KeyBytes: mKeySize + vKeySize, // The intent is still there, but this time with mVal2Size, and a zero vValSize. - ValBytes: m2ValSize, // 10+46 = 56 + ValBytes: m2ValSize + vVal2Size, IntentAge: 0, IntentCount: 1, // still there SeparatedIntentCount: 1, - IntentBytes: vKeySize, // still there, but now without vValSize - GCBytesAge: 0, // this was once erroneously negative + IntentBytes: vKeySize + vVal2Size, // still there, but now without vValSize + GCBytesAge: 0, // this was once erroneously negative } assertEq(t, engine, "after deleting", aggMS, &expAggMS) @@ -496,8 +525,6 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { t.Fatal(err) } - value := roachpb.MakeValueFromString("value") - mKeySize := int64(mvccKey(key).EncodedSize()) require.EqualValues(t, mKeySize, 2) @@ -512,8 +539,10 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { vKeySize := MVCCVersionTimestampSize require.EqualValues(t, vKeySize, 12) - vValSize := int64(len(value.RawBytes)) - require.EqualValues(t, vValSize, 10) + vVal1Size := int64(0) // tombstone + if disableSimpleValueEncoding { + vVal1Size = emptyMVCCValueHeaderSize // 7 + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, @@ -521,12 +550,12 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { LiveCount: 0, KeyBytes: mKeySize + vKeySize, // 2 + 12 = 24 KeyCount: 1, - ValBytes: mVal1Size, // 46[+2] + ValBytes: mVal1Size + vVal1Size, // 46[+2] [+7] ValCount: 1, IntentAge: 0, IntentCount: 1, SeparatedIntentCount: 1, - IntentBytes: vKeySize, // 12 + IntentBytes: vKeySize + vVal1Size, // 12 [+7] GCBytesAge: 0, } assertEq(t, engine, "after delete", aggMS, &expMS) @@ -541,14 +570,29 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { // Annoyingly, the new meta value is actually a little larger thanks to the // sequence number. Also the value is larger because the previous intent on the // transaction is recorded in the IntentHistory. - m2ValSize := int64((&enginepb.MVCCMetadata{ + encVal1, err := EncodeMVCCValue(MVCCValue{Value: roachpb.Value{RawBytes: []byte{}}}) + require.NoError(t, err) + mVal2Size := int64((&enginepb.MVCCMetadata{ Timestamp: ts2.ToLegacyTimestamp(), Txn: &txn.TxnMeta, IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: []byte{}}, + {Sequence: 0, Value: encVal1}, }, }).Size()) - require.EqualValues(t, m2ValSize, 54) + expMVal2Size := 54 + if disableSimpleValueEncoding { + expMVal2Size = 61 + } + require.EqualValues(t, mVal2Size, expMVal2Size) + + value := roachpb.MakeValueFromString("value") + + vVal2Size := int64(len(value.RawBytes)) + require.EqualValues(t, vVal2Size, 10) + if disableSimpleValueEncoding { + vVal2Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal2Size, 17) + } if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { t.Fatal(err) @@ -556,7 +600,7 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 2e9, - LiveBytes: mKeySize + m2ValSize + vKeySize + vValSize, // 2+46+12+10 = 70 + LiveBytes: mKeySize + mVal2Size + vKeySize + vVal2Size, // 2+46+12+(10[+7]) = 70[+7] LiveCount: 1, KeyCount: 1, ValCount: 1, @@ -564,12 +608,12 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { // One versioned key counts for vKeySize. KeyBytes: mKeySize + vKeySize, // The intent is still there, but this time with mVal2Size, and a zero vValSize. - ValBytes: vValSize + m2ValSize, // 10+46 = 56 + ValBytes: vVal2Size + mVal2Size, // (10[+7])+46 = 56[+7] IntentAge: 0, IntentCount: 1, // still there SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // still there, now bigger - GCBytesAge: 0, // this was once erroneously negative + IntentBytes: vKeySize + vVal2Size, // still there, now bigger + GCBytesAge: 0, // this was once erroneously negative } assertEq(t, engine, "after put", aggMS, &expAggMS) @@ -611,17 +655,22 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { vKeySize := MVCCVersionTimestampSize require.EqualValues(t, vKeySize, 12) + vValSize := int64(0) // tombstone + if disableSimpleValueEncoding { + vValSize = emptyMVCCValueHeaderSize // 7 + } + expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, KeyBytes: mKeySize + vKeySize, KeyCount: 1, - ValBytes: 0, + ValBytes: vValSize, ValCount: 1, } assertEq(t, engine, "after non-transactional delete", aggMS, &expMS) - // Write an tombstone intent at t=2s. + // Write a tombstone intent at t=2s. txn := &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, ReadTimestamp: ts2, @@ -643,13 +692,13 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { LastUpdateNanos: 2e9, KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 KeyCount: 1, - ValBytes: mValSize, // 46[+2] + ValBytes: mValSize + 2*vValSize, // 46[+2] [+7] ValCount: 2, IntentCount: 1, SeparatedIntentCount: 1, - IntentBytes: vKeySize, // TBD + IntentBytes: vKeySize + vValSize, // The original non-transactional write (at 1s) has now aged one second. - GCBytesAge: 1 * vKeySize, + GCBytesAge: 1 * (vKeySize + vValSize), } assertEq(t, engine, "after put", aggMS, &expMS) @@ -672,21 +721,21 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { // The initial write used the simple MVCCValue encoding. When resolved to // a higher timestamp, the MVCCValue retained its local timestamp, meaning // that it now uses the extended MVCCValue encoding. - vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts2)} - vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 - vValSize := vValHeaderSize + 0 // tombstone, so just a header + vVal2Header := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts2)} + vVal2HeaderSize := extendedPreludeSize + int64(vVal2Header.Size()) // 13 + vVal2Size := vVal2HeaderSize + 0 // tombstone, so just a header expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 3e9, KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 KeyCount: 1, - ValBytes: vValSize, + ValBytes: vValSize + vVal2Size, ValCount: 2, IntentCount: 0, IntentBytes: 0, // The very first write picks up another second of age. Before a bug fix, // this was failing to do so. - GCBytesAge: 2 * vKeySize, + GCBytesAge: 2 * (vKeySize + vValSize), } assertEq(t, engine, "after committing", &aggMS, &expAggMS) @@ -709,14 +758,14 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { LastUpdateNanos: 3e9, KeyBytes: mKeySize + vKeySize, // 2+12 = 14 KeyCount: 1, - ValBytes: 0, + ValBytes: vValSize, ValCount: 1, IntentCount: 0, IntentBytes: 0, // We aborted our intent, but the value we first wrote was a tombstone, and // so it's expected to retain its age. Since it's now the only value, it // also contributes as a meta key. - GCBytesAge: 2 * (mKeySize + vKeySize), + GCBytesAge: 2 * (mKeySize + vKeySize + vValSize), } assertEq(t, engine, "after aborting", &aggMS, &expAggMS) @@ -765,6 +814,9 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { vValSize := int64(len(value.RawBytes)) require.EqualValues(t, vValSize, 10) + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize // 17 + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, @@ -795,15 +847,20 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { require.EqualValues(t, mValSize, 46) mValSize += 2 + vDelSize := int64(0) // tombstone + if disableSimpleValueEncoding { + vDelSize = emptyMVCCValueHeaderSize // 7 + } + expMS = enginepb.MVCCStats{ LastUpdateNanos: 2e9, KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 KeyCount: 1, - ValBytes: mValSize + vValSize, // 46[+2]+10 = 56[+2] + ValBytes: mValSize + vValSize + vDelSize, // 46[+2]+10[+7] = 56[+2][+7] ValCount: 2, IntentCount: 1, SeparatedIntentCount: 1, - IntentBytes: vKeySize, // 12 + IntentBytes: vKeySize + vDelSize, // 12[+7] // The original non-transactional write becomes non-live at 2s, so no age // is accrued yet. GCBytesAge: 0, @@ -862,6 +919,10 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { val2 := roachpb.MakeValueFromString("longvalue") vVal2Size := int64(len(val2.RawBytes)) require.EqualValues(t, vVal2Size, 14) + if disableSimpleValueEncoding { + vVal2Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal2Size, 21) + } txn.WriteTimestamp.Forward(ts3) if err := MVCCPut(ctx, engine, &aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { @@ -870,24 +931,29 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { // Annoyingly, the new meta value is actually a little larger thanks to the // sequence number. - m2ValSizeWithHistory := int64((&enginepb.MVCCMetadata{ + encDel, err := EncodeMVCCValue(MVCCValue{Value: roachpb.Value{RawBytes: []byte{}}}) + require.NoError(t, err) + mVal2SizeWithHistory := int64((&enginepb.MVCCMetadata{ Timestamp: ts3.ToLegacyTimestamp(), Txn: &txn.TxnMeta, IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: []byte{}}, + {Sequence: 0, Value: encDel}, }, }).Size()) - - require.EqualValues(t, m2ValSizeWithHistory, 54) + expMVal2Size := 54 + if disableSimpleValueEncoding { + expMVal2Size = 61 + } + require.EqualValues(t, mVal2SizeWithHistory, expMVal2Size) expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 3e9, KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 KeyCount: 1, - ValBytes: m2ValSizeWithHistory + vValSize + vVal2Size, + ValBytes: mVal2SizeWithHistory + vValSize + vVal2Size, ValCount: 2, LiveCount: 1, - LiveBytes: mKeySize + m2ValSizeWithHistory + vKeySize + vVal2Size, + LiveBytes: mKeySize + mVal2SizeWithHistory + vKeySize + vVal2Size, IntentCount: 1, SeparatedIntentCount: 1, IntentBytes: vKeySize + vVal2Size, @@ -932,13 +998,18 @@ func TestMVCCStatsDelDelGC(t *testing.T) { mKeySize := int64(mvccKey(key).EncodedSize()) // 2 vKeySize := MVCCVersionTimestampSize // 12 + vValSize := int64(0) // tombstone + if disableSimpleValueEncoding { + vValSize = emptyMVCCValueHeaderSize // 7 + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 2e9, KeyBytes: mKeySize + 2*vKeySize, // 26 + ValBytes: 2 * vValSize, KeyCount: 1, ValCount: 2, - GCBytesAge: 1 * vKeySize, // first tombstone, aged from ts1 to ts2 + GCBytesAge: 1 * (vKeySize + vValSize), // first tombstone, aged from ts1 to ts2 } assertEq(t, engine, "after two puts", aggMS, &expMS) @@ -1016,18 +1087,21 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { m1ValSize += 2 vKeySize := MVCCVersionTimestampSize // 12 vValSize := int64(len(value.RawBytes)) // 10 + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize // 17 + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 2e9 + 1, - LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(46[+2])+12+10 = 68[+2] + LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 68[+2][+7] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 14 KeyCount: 1, - ValBytes: m1ValSize + vValSize, // (46[+2])+10 = 54[+2] + ValBytes: m1ValSize + vValSize, // (46[+2])+(10[+7]) = 54[+2][+7] ValCount: 1, IntentCount: 1, SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // 12+10 = 22 + IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] } assertEq(t, engine, "after first put", aggMS, &expMS) @@ -1042,11 +1116,13 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { // Annoyingly, the new meta value is actually a little larger thanks to the // sequence number. + encValue, err := EncodeMVCCValue(MVCCValue{Value: value}) + require.NoError(t, err) m2ValSize := int64((&enginepb.MVCCMetadata{ // 46 Timestamp: ts201.ToLegacyTimestamp(), Txn: &txn.TxnMeta, IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: value.RawBytes}, + {Sequence: 0, Value: encValue}, }, }).Size()) if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { @@ -1059,15 +1135,15 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { IntentAge: 0, LastUpdateNanos: 2e9 + 1, - LiveBytes: mKeySize + m2ValSize + vKeySize + vValSize, // 2+46+12+10 = 70 + LiveBytes: mKeySize + m2ValSize + vKeySize + vValSize, // 2+(46[+7])+12+(10[+7]) = 70[+14] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 14 KeyCount: 1, - ValBytes: m2ValSize + vValSize, // 46+10 = 56 + ValBytes: m2ValSize + vValSize, // (46[+7])+(10[+7]) = 56[+14] ValCount: 1, IntentCount: 1, SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // 12+10 = 22 + IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] } assertEq(t, engine, "after second put", aggMS, &expAggMS) @@ -1109,15 +1185,19 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { vValSize := int64(len(val1.RawBytes)) require.EqualValues(t, vValSize, 10) + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize + require.EqualValues(t, vValSize, 17) + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, KeyCount: 1, KeyBytes: mKeySize + vKeySize, // 2+12 = 14 ValCount: 1, - ValBytes: vValSize, // 10 + ValBytes: vValSize, // 10[+7] LiveCount: 1, - LiveBytes: mKeySize + vKeySize + vValSize, // 2+12+10 = 24 + LiveBytes: mKeySize + vKeySize + vValSize, // 2+12+(10[+7]) = 24[+7] } assertEq(t, engine, "after first put", aggMS, &expMS) @@ -1127,11 +1207,16 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { t.Fatal(err) } + vVal2Size := int64(0) // tombstone + if disableSimpleValueEncoding { + vVal2Size = emptyMVCCValueHeaderSize // 7 + } + expMS = enginepb.MVCCStats{ LastUpdateNanos: 2e9, KeyCount: 1, KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 - ValBytes: vValSize, // 10 + ValBytes: vValSize + vVal2Size, // 10[+7] ValCount: 2, LiveBytes: 0, LiveCount: 0, @@ -1151,7 +1236,7 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { LastUpdateNanos: 2e9, KeyCount: 1, KeyBytes: mKeySize + vKeySize, // 2+12 = 14 - ValBytes: 0, + ValBytes: vVal2Size, ValCount: 1, LiveBytes: 0, LiveCount: 0, @@ -1210,14 +1295,22 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { vVal1Size := int64(len(val1.RawBytes)) require.EqualValues(t, vVal1Size, 10) + if disableSimpleValueEncoding { + vVal1Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal1Size, 17) + } val2 := roachpb.MakeValueFromString("longvalue") vVal2Size := int64(len(val2.RawBytes)) require.EqualValues(t, vVal2Size, 14) + if disableSimpleValueEncoding { + vVal2Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal2Size, 21) + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, - SysBytes: mKeySize + mValSize + vKeySize + vVal1Size, // 11+(46[+2])+12+10 = 79[+2] + SysBytes: mKeySize + mValSize + vKeySize + vVal1Size, // 11+(46[+2])+12+(10[+7]) = 79[+2][+7] SysCount: 1, } assertEqLocal(t, engine, "after first put", aggMS, &expMS) @@ -1229,15 +1322,21 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { // The new meta value grows because we've bumped `txn.Sequence`. // The value also grows as the older value is part of the same // transaction and so contributes to the intent history. + encVal1, err := EncodeMVCCValue(MVCCValue{Value: val1}) + require.NoError(t, err) mVal2Size := int64((&enginepb.MVCCMetadata{ Timestamp: ts2.ToLegacyTimestamp(), Deleted: false, Txn: &txn.TxnMeta, IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: val1.RawBytes}, + {Sequence: 0, Value: encVal1}, }, }).Size()) - require.EqualValues(t, mVal2Size, 64) + expMVal2Size := 64 + if disableSimpleValueEncoding { + expMVal2Size = 71 + } + require.EqualValues(t, mVal2Size, expMVal2Size) if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { t.Fatal(err) @@ -1245,7 +1344,7 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { expMS = enginepb.MVCCStats{ LastUpdateNanos: 1e9, - SysBytes: mKeySize + mVal2Size + vKeySize + vVal2Size, // 11+46+12+14 = 83 + SysBytes: mKeySize + mVal2Size + vKeySize + vVal2Size, // 11+(46[+7])+12+14 = 83[+7] SysCount: 1, } @@ -1300,14 +1399,22 @@ func TestMVCCStatsTxnSysPutAbort(t *testing.T) { vVal1Size := int64(len(val1.RawBytes)) require.EqualValues(t, vVal1Size, 10) + if disableSimpleValueEncoding { + vVal1Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal1Size, 17) + } val2 := roachpb.MakeValueFromString("longvalue") vVal2Size := int64(len(val2.RawBytes)) require.EqualValues(t, vVal2Size, 14) + if disableSimpleValueEncoding { + vVal2Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal2Size, 21) + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, - SysBytes: mKeySize + mValSize + vKeySize + vVal1Size, // 11+(46[+2])+12+10 = 79[+2] + SysBytes: mKeySize + mValSize + vKeySize + vVal1Size, // 11+(46[+2])+12+(10[+7]) = 79[+2][+7] SysCount: 1, } assertEqLocal(t, engine, "after first put", aggMS, &expMS) @@ -1362,14 +1469,22 @@ func TestMVCCStatsSysPutPut(t *testing.T) { vVal1Size := int64(len(val1.RawBytes)) require.EqualValues(t, vVal1Size, 10) + if disableSimpleValueEncoding { + vVal1Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal1Size, 17) + } val2 := roachpb.MakeValueFromString("longvalue") vVal2Size := int64(len(val2.RawBytes)) require.EqualValues(t, vVal2Size, 14) + if disableSimpleValueEncoding { + vVal2Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal2Size, 21) + } expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, - SysBytes: mKeySize + vKeySize + vVal1Size, // 11+12+10 = 33 + SysBytes: mKeySize + vKeySize + vVal1Size, // 11+12+(10[+7]) = 33[+7] SysCount: 1, } assertEqLocal(t, engine, "after first put", aggMS, &expMS) diff --git a/pkg/storage/mvcc_value.go b/pkg/storage/mvcc_value.go index 587a42f0973f..2edb4819b611 100644 --- a/pkg/storage/mvcc_value.go +++ b/pkg/storage/mvcc_value.go @@ -16,6 +16,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -136,10 +138,40 @@ func (v MVCCValue) SafeFormat(w redact.SafePrinter, _ rune) { w.Print(v.Value.PrettyPrint()) } +// When running a metamorphic build, disable the simple MVCC value encoding to +// prevent code from assuming that the MVCCValue encoding is identical to the +// roachpb.Value encoding. +var disableSimpleValueEncoding = util.ConstantWithMetamorphicTestBool( + "mvcc-value-disable-simple-encoding", false) + +// SkipIfSimpleValueEncodingDisabled skips this test during metamorphic runs +// that have disabled the simple MVCC value encoding. +func SkipIfSimpleValueEncodingDisabled(t skip.SkippableTest) { + t.Helper() + if disableSimpleValueEncoding { + skip.IgnoreLint(t, "disabled under metamorphic") + } +} + +var emptyValueHeader = func() enginepb.MVCCValueHeader { + var h enginepb.MVCCValueHeader + // Hacky: we don't have room in the mid-stack inlining budget in either + // encodedMVCCValueSize or EncodeMVCCValue to add to the simple encoding + // condition (e.g. `&& !disableSimpleValueEncoding`). So to have the same + // effect, we replace the empty value header with a header we never expect + // to see. We never expect LocalTimestamp to be set to MaxClockTimestamp + // because if it was set to that value, LocalTimestampNeeded would never + // return true. + if disableSimpleValueEncoding { + h.LocalTimestamp = hlc.MaxClockTimestamp + } + return h +}() + // encodedMVCCValueSize returns the size of the MVCCValue when encoded. //gcassert:inline func encodedMVCCValueSize(v MVCCValue) int { - if v.MVCCValueHeader == (enginepb.MVCCValueHeader{}) { + if v.MVCCValueHeader == emptyValueHeader { return len(v.Value.RawBytes) } return extendedPreludeSize + v.MVCCValueHeader.Size() + len(v.Value.RawBytes) @@ -149,7 +181,7 @@ func encodedMVCCValueSize(v MVCCValue) int { // comment on MVCCValue for a description of the encoding scheme. //gcassert:inline func EncodeMVCCValue(v MVCCValue) ([]byte, error) { - if v.MVCCValueHeader == (enginepb.MVCCValueHeader{}) { + if v.MVCCValueHeader == emptyValueHeader { // Simple encoding. Use the roachpb.Value encoding directly with no // modification. No need to re-allocate or copy. return v.Value.RawBytes, nil diff --git a/pkg/storage/mvcc_value_test.go b/pkg/storage/mvcc_value_test.go index fc8ee3473009..ee6ce7f1b7f7 100644 --- a/pkg/storage/mvcc_value_test.go +++ b/pkg/storage/mvcc_value_test.go @@ -110,6 +110,7 @@ func TestMVCCValueFormat(t *testing.T) { func TestEncodeDecodeMVCCValue(t *testing.T) { defer leaktest.AfterTest(t)() + SkipIfSimpleValueEncodingDisabled(t) var strVal, intVal roachpb.Value strVal.SetString("foo") diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 0fcf6fac653e..d16eb8e30c70 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -1214,6 +1214,7 @@ func TestPebbleMVCCTimeIntervalCollectorAndFilter(t *testing.T) { func TestPebbleFlushCallbackAndDurabilityRequirement(t *testing.T) { defer leaktest.AfterTest(t)() + SkipIfSimpleValueEncodingDisabled(t) eng := createTestPebbleEngine() defer eng.Close() diff --git a/pkg/storage/sst.go b/pkg/storage/sst.go index 35672f4b1fca..5ddb724ee458 100644 --- a/pkg/storage/sst.go +++ b/pkg/storage/sst.go @@ -74,8 +74,8 @@ func CheckSSTConflicts( return enginepb.MVCCStats{}, err } - extKey, extValue := extIter.UnsafeKey(), extIter.UnsafeValue() - sstKey, sstValue := sstIter.UnsafeKey(), sstIter.UnsafeValue() + extKey, extValueRaw := extIter.UnsafeKey(), extIter.UnsafeValue() + sstKey, sstValueRaw := sstIter.UnsafeKey(), sstIter.UnsafeValue() // Keep seeking the iterators until both keys are equal. if cmp := bytes.Compare(extKey.Key, sstKey.Key); cmp < 0 { @@ -94,7 +94,14 @@ func CheckSSTConflicts( if !sstKey.IsValue() { return enginepb.MVCCStats{}, errors.New("SST keys must have timestamps") } - if len(sstValue) == 0 { + sstValue, ok, err := tryDecodeSimpleMVCCValue(sstValueRaw) + if !ok && err == nil { + sstValue, err = decodeExtendedMVCCValue(sstValueRaw) + } + if err != nil { + return enginepb.MVCCStats{}, err + } + if sstValue.IsTombstone() { return enginepb.MVCCStats{}, errors.New("SST values cannot be tombstones") } if !extKey.IsValue() { @@ -125,6 +132,13 @@ func CheckSSTConflicts( continue } } + extValue, ok, err := tryDecodeSimpleMVCCValue(extValueRaw) + if !ok && err == nil { + extValue, err = decodeExtendedMVCCValue(extValueRaw) + } + if err != nil { + return enginepb.MVCCStats{}, err + } // Allow certain idempotent writes where key/timestamp/value all match: // @@ -133,7 +147,7 @@ func CheckSSTConflicts( allowIdempotent := (!disallowShadowingBelow.IsEmpty() && disallowShadowingBelow.LessEq(extKey.Timestamp)) || (disallowShadowingBelow.IsEmpty() && disallowShadowing) if allowIdempotent && sstKey.Timestamp.Equal(extKey.Timestamp) && - bytes.Equal(extValue, sstValue) { + bytes.Equal(extValueRaw, sstValueRaw) { // This SST entry will effectively be a noop, but its stats have already // been accounted for resulting in double-counting. To address this we // send back a stats diff for these existing KVs so that we can subtract @@ -151,10 +165,10 @@ func CheckSSTConflicts( statsDiff.KeyCount-- // Update the stats to account for the skipped versioned key/value. - totalBytes = int64(len(sstValue)) + MVCCVersionTimestampSize + totalBytes = int64(len(sstValueRaw)) + MVCCVersionTimestampSize statsDiff.LiveBytes -= totalBytes statsDiff.KeyBytes -= MVCCVersionTimestampSize - statsDiff.ValBytes -= int64(len(sstValue)) + statsDiff.ValBytes -= int64(len(sstValueRaw)) statsDiff.ValCount-- sstIter.NextKey() @@ -171,9 +185,9 @@ func CheckSSTConflicts( // a WriteTooOldError -- that error implies that the client should // retry at a higher timestamp, but we already know that such a retry // would fail (because it will shadow an existing key). - if len(extValue) > 0 && (!disallowShadowingBelow.IsEmpty() || disallowShadowing) { + if !extValue.IsTombstone() && (!disallowShadowingBelow.IsEmpty() || disallowShadowing) { allowShadow := !disallowShadowingBelow.IsEmpty() && - disallowShadowingBelow.LessEq(extKey.Timestamp) && bytes.Equal(extValue, sstValue) + disallowShadowingBelow.LessEq(extKey.Timestamp) && bytes.Equal(extValueRaw, sstValueRaw) if !allowShadow { return enginepb.MVCCStats{}, errors.Errorf( "ingested key collides with an existing one: %s", sstKey.Key) @@ -194,10 +208,10 @@ func CheckSSTConflicts( // to take into account the existing KV pair. statsDiff.KeyCount-- statsDiff.KeyBytes -= int64(len(extKey.Key) + 1) - if len(extValue) > 0 { + if !extValue.IsTombstone() { statsDiff.LiveCount-- statsDiff.LiveBytes -= int64(len(extKey.Key) + 1) - statsDiff.LiveBytes -= int64(len(extValue)) + MVCCVersionTimestampSize + statsDiff.LiveBytes -= int64(len(extValueRaw)) + MVCCVersionTimestampSize } sstIter.NextKey() diff --git a/pkg/storage/sst_iterator.go b/pkg/storage/sst_iterator.go index 9bd8b49b1b39..a10951a7e483 100644 --- a/pkg/storage/sst_iterator.go +++ b/pkg/storage/sst_iterator.go @@ -13,7 +13,6 @@ package storage import ( "bytes" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/sstable" "github.com/cockroachdb/pebble/vfs" @@ -107,7 +106,7 @@ func (r *sstIterator) SeekGE(key MVCCKey) { r.err = r.iter.Error() } if r.iterValid && r.err == nil && r.verify && r.mvccKey.IsValue() { - r.err = roachpb.Value{RawBytes: r.value}.Verify(r.mvccKey.Key) + r.verifyValue() } r.prevSeekKey.Key = append(r.prevSeekKey.Key[:0], r.mvccKey.Key...) r.prevSeekKey.Timestamp = r.mvccKey.Timestamp @@ -134,7 +133,7 @@ func (r *sstIterator) Next() { r.err = r.iter.Error() } if r.iterValid && r.err == nil && r.verify && r.mvccKey.IsValue() { - r.err = roachpb.Value{RawBytes: r.value}.Verify(r.mvccKey.Key) + r.verifyValue() } } @@ -158,3 +157,16 @@ func (r *sstIterator) UnsafeKey() MVCCKey { func (r *sstIterator) UnsafeValue() []byte { return r.value } + +// verifyValue verifies the checksum of the current value. +func (r *sstIterator) verifyValue() { + mvccValue, ok, err := tryDecodeSimpleMVCCValue(r.value) + if !ok && err == nil { + mvccValue, err = decodeExtendedMVCCValue(r.value) + } + if err != nil { + r.err = err + } else { + r.err = mvccValue.Value.Verify(r.mvccKey.Key) + } +}