diff --git a/pkg/kv/kvserver/batcheval/declare.go b/pkg/kv/kvserver/batcheval/declare.go index 0ec8fae8b8bf..101e1076ddeb 100644 --- a/pkg/kv/kvserver/batcheval/declare.go +++ b/pkg/kv/kvserver/batcheval/declare.go @@ -15,6 +15,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/keys" + "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" @@ -52,24 +53,29 @@ func DefaultDeclareIsolatedKeys( timestamp := header.Timestamp if roachpb.IsReadOnly(req) && !roachpb.IsLocking(req) { access = spanset.SpanReadOnly - if header.Txn != nil { - // For transactional reads, acquire read latches all the way up to - // the transaction's uncertainty limit, because reads may observe - // writes all the way up to this timestamp. - // - // It is critical that reads declare latches up through their - // uncertainty interval so that they are properly synchronized with - // earlier writes that may have a happened-before relationship with - // the read. These writes could not have completed and returned to - // the client until they were durable in the Range's Raft log. - // However, they may not have been applied to the replica's state - // machine by the time the write was acknowledged, because Raft - // entry application occurs asynchronously with respect to the - // writer (see AckCommittedEntriesBeforeApplication). Latching is - // the only mechanism that ensures that any observers of the write - // wait for the write apply before reading. - timestamp.Forward(header.Txn.GlobalUncertaintyLimit) - } + + // For non-locking reads, acquire read latches all the way up to the + // request's worst-case (i.e. global) uncertainty limit, because reads may + // observe writes all the way up to this timestamp. + // + // It is critical that reads declare latches up through their uncertainty + // interval so that they are properly synchronized with earlier writes that + // may have a happened-before relationship with the read. These writes could + // not have completed and returned to the client until they were durable in + // the Range's Raft log. However, they may not have been applied to the + // replica's state machine by the time the write was acknowledged, because + // Raft entry application occurs asynchronously with respect to the writer + // (see AckCommittedEntriesBeforeApplication). Latching is the only + // mechanism that ensures that any observers of the write wait for the write + // apply before reading. + // + // NOTE: we pass an empty lease status here, which means that observed + // timestamps collected by transactions will not be used. The actual + // uncertainty interval used by the request may be smaller (i.e. contain a + // local limit), but we can't determine that until after we have declared + // keys, acquired latches, and consulted the replica's lease. + in := uncertainty.ComputeInterval(header, kvserverpb.LeaseStatus{}, maxOffset) + timestamp.Forward(in.GlobalLimit) } latchSpans.AddMVCC(access, req.Header().Span(), timestamp) lockSpans.AddNonMVCC(access, req.Header().Span()) diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index f52a0e42c05a..ef7404a804c1 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -701,6 +701,177 @@ func TestTxnReadWithinUncertaintyIntervalAfterLeaseTransfer(t *testing.T) { require.IsType(t, &roachpb.ReadWithinUncertaintyIntervalError{}, pErr.GetDetail()) } +// TestNonTxnReadWithinUncertaintyIntervalAfterLeaseTransfer tests a case where +// a non-transactional request defers its timestamp allocation to a replica that +// does not hold the lease at the time of receiving the request, but does by the +// time that the request consults the lease. In the test, a value is written on +// the previous leaseholder at a higher timestamp than that assigned to the non- +// transactional request. After the lease transfer, the non-txn request is +// required by uncertainty.ComputeInterval to forward its local uncertainty +// limit to the new lease start time. This prevents the read from ignoring the +// previous write, which avoids a stale read. Instead, the non-txn read hits an +// uncertainty error, performs a server-side retry, and re-evaluates with a +// timestamp above the write. +// +// This test exercises the hazard described in "reason #1" of uncertainty.D7. +func TestNonTxnReadWithinUncertaintyIntervalAfterLeaseTransfer(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + // Inject a request filter, which intercepts the server-assigned timestamp + // of a non-transactional request and then blocks that request until after + // the lease has been transferred to the server. + type nonTxnGetKey struct{} + nonTxnOrigTsC := make(chan hlc.Timestamp, 1) + nonTxnBlockerC := make(chan struct{}) + requestFilter := func(ctx context.Context, ba roachpb.BatchRequest) *roachpb.Error { + if ctx.Value(nonTxnGetKey{}) != nil { + // Give the test the server-assigned timestamp. + require.NotNil(t, ba.TimestampFromServerClock) + nonTxnOrigTsC <- ba.Timestamp + // Wait for the test to give the go-ahead. + select { + case <-nonTxnBlockerC: + case <-ctx.Done(): + } + } + return nil + } + var uncertaintyErrs int32 + concurrencyRetryFilter := func(ctx context.Context, _ roachpb.BatchRequest, pErr *roachpb.Error) { + if ctx.Value(nonTxnGetKey{}) != nil { + if _, ok := pErr.GetDetail().(*roachpb.ReadWithinUncertaintyIntervalError); ok { + atomic.AddInt32(&uncertaintyErrs, 1) + } + } + } + + const numNodes = 2 + var manuals []*hlc.HybridManualClock + 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{ + TestingRequestFilter: requestFilter, + TestingConcurrencyRetryFilter: concurrencyRetryFilter, + }, + }, + } + } + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: serverArgs, + }) + defer tc.Stopper().Stop(ctx) + + // Split off a scratch range and upreplicate to node 2. + key := tc.ScratchRange(t) + desc := tc.LookupRangeOrFatal(t, key) + tc.AddVotersOrFatal(t, key, tc.Target(1)) + + // Pause the servers' clocks going forward. + var maxNanos int64 + for _, m := range manuals { + m.Pause() + if cur := m.UnixNano(); cur > maxNanos { + maxNanos = cur + } + } + // After doing so, perfectly synchronize them. + for _, m := range manuals { + m.Increment(maxNanos - m.UnixNano()) + } + + // Initiate a non-txn read on node 2. The request will be intercepted after + // the request has received a server-assigned timestamp, but before it has + // consulted the lease. We'll transfer the lease to node 2 before the request + // checks, so that it ends up evaluating on node 2. + type resp struct { + *roachpb.BatchResponse + *roachpb.Error + } + nonTxnRespC := make(chan resp, 1) + _ = tc.Stopper().RunAsyncTask(ctx, "non-txn get", func(ctx context.Context) { + ctx = context.WithValue(ctx, nonTxnGetKey{}, "foo") + ba := roachpb.BatchRequest{} + ba.RangeID = desc.RangeID + ba.Add(getArgs(key)) + br, pErr := tc.GetFirstStoreFromServer(t, 1).Send(ctx, ba) + nonTxnRespC <- resp{br, pErr} + }) + + // Wait for the non-txn read to get stuck. + var nonTxnOrigTs hlc.Timestamp + select { + case nonTxnOrigTs = <-nonTxnOrigTsC: + case nonTxnResp := <-nonTxnRespC: + t.Fatalf("unexpected response %+v", nonTxnResp) + } + + // Advance the clock on node 1. + manuals[0].Increment(100) + + // Perform a non-txn write on node 1. This will grab a timestamp from node 1's + // clock, which leads the clock on node 2 and the timestamp assigned to the + // non-txn read. + // + // NOTE: we perform the clock increment and write _after_ sending the non-txn + // read. Ideally, we would write this test such that we did this before + // beginning the read on node 2, so that the absence of an uncertainty error + // would be a true "stale read". However, doing so causes the test to be flaky + // because background operations can leak the clock signal from node 1 to node + // 2 between the time that we write and the time that the non-txn read request + // is sent. If we had a way to disable all best-effort HLC clock stabilization + // channels and only propagate clock signals when strictly necessary then it's + // possible that we could avoid flakiness. For now, we just re-order the + // operations and assert that we observe an uncertainty error even though its + // absence would not be a true stale read. + ba := roachpb.BatchRequest{} + ba.Add(putArgs(key, []byte("val"))) + br, pErr := tc.Servers[0].DistSender().Send(ctx, ba) + require.Nil(t, pErr) + writeTs := br.Timestamp + require.True(t, nonTxnOrigTs.Less(writeTs)) + + // Then transfer the lease to node 2. The new lease should end up with a start + // time above the timestamp assigned to the non-txn read. + var lease roachpb.Lease + tc.TransferRangeLeaseOrFatal(t, desc, tc.Target(1)) + testutils.SucceedsSoon(t, func() error { + repl := tc.GetFirstStoreFromServer(t, 1).LookupReplica(desc.StartKey) + lease, _ = repl.GetLease() + if lease.Replica.NodeID != repl.NodeID() { + return errors.Errorf("expected lease transfer to node 2: %s", lease) + } + return nil + }) + require.True(t, nonTxnOrigTs.Less(lease.Start.ToTimestamp())) + + // Let the non-txn read proceed. It should complete, but only after hitting a + // ReadWithinUncertaintyInterval, performing a server-side retry, reading + // again at a higher timestamp, and returning the written value. + close(nonTxnBlockerC) + nonTxnResp := <-nonTxnRespC + require.Nil(t, nonTxnResp.Error) + br = nonTxnResp.BatchResponse + require.NotNil(t, br) + require.True(t, nonTxnOrigTs.Less(br.Timestamp)) + require.True(t, writeTs.LessEq(br.Timestamp)) + require.Len(t, br.Responses, 1) + require.NotNil(t, br.Responses[0].GetGet()) + require.NotNil(t, br.Responses[0].GetGet().Value) + require.Equal(t, writeTs, br.Responses[0].GetGet().Value.Timestamp) + require.Equal(t, int32(1), atomic.LoadInt32(&uncertaintyErrs)) +} + // TestRangeLookupUseReverse tests whether the results and the results count // are correct when scanning in reverse order. func TestRangeLookupUseReverse(t *testing.T) { diff --git a/pkg/kv/kvserver/replica_evaluate.go b/pkg/kv/kvserver/replica_evaluate.go index a9037190b5f8..5c5691716171 100644 --- a/pkg/kv/kvserver/replica_evaluate.go +++ b/pkg/kv/kvserver/replica_evaluate.go @@ -574,9 +574,8 @@ func canDoServersideRetry( case *roachpb.WriteTooOldError: newTimestamp = tErr.RetryTimestamp() - // TODO(nvanbenschoten): give non-txn requests uncertainty intervals. #73732. - //case *roachpb.ReadWithinUncertaintyIntervalError: - // newTimestamp = tErr.RetryTimestamp() + case *roachpb.ReadWithinUncertaintyIntervalError: + newTimestamp = tErr.RetryTimestamp() default: return false diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index 21cf12b33c61..fb4df085fee9 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -46,7 +46,7 @@ func (r *Replica) executeReadOnlyBatch( // Compute the transaction's local uncertainty limit using observed // timestamps, which can help avoid uncertainty restarts. - ui := uncertainty.ComputeInterval(ba.Txn, st) + ui := uncertainty.ComputeInterval(&ba.Header, st, r.Clock().MaxOffset()) // Evaluate read-only batch command. rec := NewReplicaEvalContext(r, g.LatchSpans()) diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index cc27fc60df18..8d388c1e2ce6 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -808,21 +808,23 @@ func (r *Replica) handleReadWithinUncertaintyIntervalError( if !canDoServersideRetry(ctx, pErr, ba, nil, nil, nil) { return nil, pErr } - // TODO(nvanbenschoten): give non-txn requests uncertainty intervals. #73732. - //if ba.Txn == nil && ba.Timestamp.Synthetic { - // // 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 - // // for transactional requests because they will do this during their - // // commit-wait sleep after committing. - // // - // // See TxnCoordSender.maybeCommitWait for a discussion about why doing this - // // is necessary to preserve real-time ordering for transactions that write - // // into the future. - // if err := r.Clock().SleepUntil(ctx, ba.Timestamp); err != nil { - // return nil, roachpb.NewError(err) - // } - //} + if ba.Txn == nil && ba.Timestamp.Synthetic { + // 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 + // for transactional requests because they will do this during their + // commit-wait sleep after committing. + // + // See TxnCoordSender.maybeCommitWait for a discussion about why doing this + // is necessary to preserve real-time ordering for transactions that write + // into the future. + var cancel func() + ctx, cancel = r.store.Stopper().WithCancelOnQuiesce(ctx) + defer cancel() + if err := r.Clock().SleepUntil(ctx, ba.Timestamp); err != nil { + return nil, roachpb.NewError(err) + } + } return ba, nil } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index fdc02a2aa41d..2c0a89632fc4 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -10696,6 +10696,27 @@ func TestReplicaServersideRefreshes(t *testing.T) { // EndTransaction. This is hard to do at the moment, though, because we // never defer the handling of the write too old conditions to the end of // the transaction (but we might in the future). + { + name: "serverside-refresh of read within uncertainty interval error on get in non-txn", + setupFn: func() (hlc.Timestamp, error) { + return put("a", "put") + }, + batchFn: func(ts hlc.Timestamp) (ba roachpb.BatchRequest, expTS hlc.Timestamp) { + ba.Timestamp = ts.Prev() + // NOTE: set the TimestampFromServerClock field manually. This is + // usually set on the server for non-transactional requests without + // client-assigned timestamps. It is also usually set to the same + // value as the server-assigned timestamp. But to have more control + // over the uncertainty interval that this request receives, we set + // it here to a value above the request timestamp. + serverTS := ts.Next() + ba.TimestampFromServerClock = (*hlc.ClockTimestamp)(&serverTS) + expTS = ts.Next() + get := getArgs(roachpb.Key("a")) + ba.Add(&get) + return + }, + }, { name: "serverside-refresh of read within uncertainty interval error on get in non-1PC txn", setupFn: func() (hlc.Timestamp, error) { diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 276a07aef347..681395c07e29 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -96,7 +96,7 @@ func (r *Replica) executeWriteBatch( // Compute the transaction's local uncertainty limit using observed // timestamps, which can help avoid uncertainty restarts. - ui := uncertainty.ComputeInterval(ba.Txn, st) + ui := uncertainty.ComputeInterval(&ba.Header, st, r.Clock().MaxOffset()) // Start tracking this request if it is an MVCC write (i.e. if it's the kind // of request that needs to obey the closed timestamp). The act of tracking diff --git a/pkg/kv/kvserver/uncertainty/compute.go b/pkg/kv/kvserver/uncertainty/compute.go index 5e1c6a4f6f3b..6d9827c4028b 100644 --- a/pkg/kv/kvserver/uncertainty/compute.go +++ b/pkg/kv/kvserver/uncertainty/compute.go @@ -11,12 +11,15 @@ package uncertainty import ( + "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" ) -// ComputeInterval returns the provided transaction's uncertainty interval to be -// used when evaluating requests under the specified lease. +// ComputeInterval returns the provided request's uncertainty interval to be +// used when evaluating under the specified lease. // // The computation uses observed timestamps gathered from the leaseholder's node // to limit the interval's local uncertainty limit. This prevents unnecessary @@ -55,16 +58,26 @@ import ( // right-hand side. // TODO(nvanbenschoten): fix this bug with range merges. // -func ComputeInterval(txn *roachpb.Transaction, status kvserverpb.LeaseStatus) Interval { - // Non-transactional requests do not have uncertainty intervals. - // TODO(nvanbenschoten): Yet, they should. Fix this. - // See https://github.com/cockroachdb/cockroach/issues/58459. - if txn == nil { - return Interval{} +func ComputeInterval( + h *roachpb.Header, status kvserverpb.LeaseStatus, maxOffset time.Duration, +) Interval { + if h.Txn != nil { + return computeIntervalForTxn(h.Txn, status) + } + return computeIntervalForNonTxn(h, status, maxOffset) +} + +func computeIntervalForTxn(txn *roachpb.Transaction, status kvserverpb.LeaseStatus) Interval { + in := Interval{ + // The transaction's global uncertainty limit is computed by its coordinator + // when the transaction is initiated. It stays constant across all requests + // issued by the transaction and across all retries. + GlobalLimit: txn.GlobalUncertaintyLimit, } - in := Interval{GlobalLimit: txn.GlobalUncertaintyLimit} if status.State != kvserverpb.LeaseState_VALID { + // If the lease is invalid, this must be a follower read. In such cases, we + // must use the most pessimistic uncertainty limit. return in } @@ -83,14 +96,82 @@ func ComputeInterval(txn *roachpb.Transaction, status kvserverpb.LeaseStatus) In } in.LocalLimit = obsTs - // If the lease is valid, we use the greater of the observed timestamp and - // the lease start time. This ensures we avoid incorrect assumptions about - // when data was written, in absolute time on a different node, which held - // the lease before this replica acquired it. - in.LocalLimit.Forward(status.Lease.Start) + // Adjust the uncertainty interval for the lease it is being used under. + in.LocalLimit.Forward(minimumLocalLimitForLeaseholder(status.Lease)) + + // The local uncertainty limit should always be <= the global uncertainty + // limit. + in.LocalLimit.BackwardWithTimestamp(in.GlobalLimit) + return in +} + +func computeIntervalForNonTxn( + h *roachpb.Header, status kvserverpb.LeaseStatus, maxOffset time.Duration, +) Interval { + if h.TimestampFromServerClock == nil || h.ReadConsistency != roachpb.CONSISTENT { + // Non-transactional requests with client-provided timestamps do not + // guarantee linearizability. Neither do entirely inconsistent requests. + // As a result, they do not have uncertainty intervals. + return Interval{} + } + + // Non-transactional requests that defer their timestamp allocation to the + // leaseholder of their (single) range do have uncertainty intervals. As a + // result, they do guarantee linearizability. + in := Interval{ + // Even though the non-transactional request received its timestamp from the + // leaseholder of its range, it can still observe writes that were performed + // before it in real-time that have MVCC timestamps above its timestamp. In + // these cases, it needs to perform an uncertainty restart. + // + // For example, the non-transactional request may observe an intent with a + // provisional timestamp below its server-assigned timestamp. It will begin + // waiting on this intent. It is possible for the intent to then be resolved + // (asynchronously with respect to the intent's txn commit) with a timestamp + // above its server-assigned timestamp. To guarantee linearizability, the + // non-transactional request must observe the effect of the intent write, so + // it must perform a (server-side) uncertainty restart to a timestamp above + // the now-resolved write. + // + // See the comment on D7 in doc.go for an example. + GlobalLimit: h.TimestampFromServerClock.ToTimestamp().Add(maxOffset.Nanoseconds(), 0), + } + + if status.State != kvserverpb.LeaseState_VALID { + // If the lease is invalid, this is either a lease request or we are computing + // the request's uncertainty interval before grabbing latches and checking for + // the current lease. Either way, return without a local limit. + return in + } + + // The request's timestamp was selected on this server, so it can serve the + // role of an observed timestamp and as the local uncertainty limit. + in.LocalLimit = *h.TimestampFromServerClock + + // Adjust the uncertainty interval for the lease it is being used under. + in.LocalLimit.Forward(minimumLocalLimitForLeaseholder(status.Lease)) // The local uncertainty limit should always be <= the global uncertainty // limit. in.LocalLimit.BackwardWithTimestamp(in.GlobalLimit) return in } + +// minimumLocalLimitForLeaseholder returns the minimum timestamp that can be +// used as a local limit when evaluating a request under the specified lease. +// See the comment on ComputeInterval for an explanation of cases where observed +// timestamps captured on the current leaseholder's node are not applicable to +// data written by prior leaseholders (either before a lease change or before a +// range merge). +func minimumLocalLimitForLeaseholder(lease roachpb.Lease) hlc.ClockTimestamp { + // If the lease is valid, we use the greater of the observed timestamp and + // the lease start time. This ensures we avoid incorrect assumptions about + // when data was written, in absolute time on a different node, which held + // the lease before this replica acquired it. + min := lease.Start + + // TODO(nvanbenschoten): handle RHS freeze timestamp after merge here when + // we fix https://github.com/cockroachdb/cockroach/issues/73292. + + return min +} diff --git a/pkg/kv/kvserver/uncertainty/compute_test.go b/pkg/kv/kvserver/uncertainty/compute_test.go index b24a2e0bf836..a3ee156f4a5b 100644 --- a/pkg/kv/kvserver/uncertainty/compute_test.go +++ b/pkg/kv/kvserver/uncertainty/compute_test.go @@ -23,11 +23,14 @@ import ( func TestComputeInterval(t *testing.T) { defer leaktest.AfterTest(t)() + const maxOffset = 10 + now := hlc.ClockTimestamp{WallTime: 15} + txn := &roachpb.Transaction{ ReadTimestamp: hlc.Timestamp{WallTime: 10}, - GlobalUncertaintyLimit: hlc.Timestamp{WallTime: 20}, + GlobalUncertaintyLimit: hlc.Timestamp{WallTime: 10 + maxOffset}, } - txn.UpdateObservedTimestamp(1, hlc.ClockTimestamp{WallTime: 15}) + txn.UpdateObservedTimestamp(1, now) repl1 := roachpb.ReplicaDescriptor{NodeID: 1} repl2 := roachpb.ReplicaDescriptor{NodeID: 2} @@ -37,19 +40,72 @@ func TestComputeInterval(t *testing.T) { } testCases := []struct { - name string - txn *roachpb.Transaction - lease kvserverpb.LeaseStatus - exp Interval + name string + txn *roachpb.Transaction + tsFromServerClock *hlc.ClockTimestamp + lease kvserverpb.LeaseStatus + exp Interval }{ { - name: "no txn", - txn: nil, - lease: lease, - exp: Interval{}, + name: "no txn, client ts", + txn: nil, + tsFromServerClock: nil, + lease: lease, + exp: Interval{}, + }, + { + name: "no txn, server ts", + txn: nil, + tsFromServerClock: &now, + lease: lease, + exp: Interval{ + GlobalLimit: hlc.Timestamp{WallTime: 25}, + LocalLimit: hlc.ClockTimestamp{WallTime: 15}, + }, + }, + { + name: "no txn, server ts, invalid lease", + txn: nil, + tsFromServerClock: &now, + lease: func() kvserverpb.LeaseStatus { + leaseClone := lease + leaseClone.State = kvserverpb.LeaseState_EXPIRED + return leaseClone + }(), + exp: Interval{ + GlobalLimit: hlc.Timestamp{WallTime: 25}, + }, + }, + { + name: "no txn, server ts, lease with start time above server ts", + txn: nil, + tsFromServerClock: &now, + lease: func() kvserverpb.LeaseStatus { + leaseClone := lease + leaseClone.Lease.Start = hlc.ClockTimestamp{WallTime: 18} + return leaseClone + }(), + exp: Interval{ + GlobalLimit: hlc.Timestamp{WallTime: 25}, + LocalLimit: hlc.ClockTimestamp{WallTime: 18}, + }, + }, + { + name: "no txn, server ts, lease with start time above server ts + max offset", + txn: nil, + tsFromServerClock: &now, + lease: func() kvserverpb.LeaseStatus { + leaseClone := lease + leaseClone.Lease.Start = hlc.ClockTimestamp{WallTime: 32} + return leaseClone + }(), + exp: Interval{ + GlobalLimit: hlc.Timestamp{WallTime: 25}, + LocalLimit: hlc.ClockTimestamp{WallTime: 25}, + }, }, { - name: "invalid lease", + name: "txn, invalid lease", txn: txn, lease: func() kvserverpb.LeaseStatus { leaseClone := lease @@ -59,7 +115,7 @@ func TestComputeInterval(t *testing.T) { exp: Interval{GlobalLimit: txn.GlobalUncertaintyLimit}, }, { - name: "no observed timestamp", + name: "txn, no observed timestamp", txn: txn, lease: func() kvserverpb.LeaseStatus { leaseClone := lease @@ -78,7 +134,7 @@ func TestComputeInterval(t *testing.T) { }, }, { - name: "valid lease with start time above observed timestamp", + name: "txn, valid lease with start time above observed timestamp", txn: txn, lease: func() kvserverpb.LeaseStatus { leaseClone := lease @@ -91,7 +147,7 @@ func TestComputeInterval(t *testing.T) { }, }, { - name: "valid lease with start time above max timestamp", + name: "txn, valid lease with start time above max timestamp", txn: txn, lease: func() kvserverpb.LeaseStatus { leaseClone := lease @@ -106,7 +162,10 @@ func TestComputeInterval(t *testing.T) { } for _, test := range testCases { t.Run(test.name, func(t *testing.T) { - require.Equal(t, test.exp, ComputeInterval(test.txn, test.lease)) + var h roachpb.Header + h.Txn = test.txn + h.TimestampFromServerClock = test.tsFromServerClock + require.Equal(t, test.exp, ComputeInterval(&h, test.lease, maxOffset)) }) } } diff --git a/pkg/kv/kvserver/uncertainty/doc.go b/pkg/kv/kvserver/uncertainty/doc.go index 30140f4298c7..09e0da47e66f 100644 --- a/pkg/kv/kvserver/uncertainty/doc.go +++ b/pkg/kv/kvserver/uncertainty/doc.go @@ -128,6 +128,7 @@ var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp // 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 // non-transactional writes. It is more complicated for transactional writes @@ -137,6 +138,7 @@ var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp // 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. +// // 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 // enforce that when a Replica acquires a lease it bumps its node's clock to a @@ -192,6 +194,116 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // D7 ———————————————————————————————————————————————— // +// TimestampFromServerClock +// +// Non-transactional requests that defer their timestamp allocation to the +// leaseholder of their (single) range also have uncertainty intervals, which +// ensures that they also guarantee single-key linearizability even with only +// loose (but bounded) clock synchronization. Non-transactional requests that +// use a client-provided timestamp do not have uncertainty intervals and do not +// make real-time ordering guarantees. +// +// Unlike transactions, which establish an uncertainty interval on their +// coordinator node during initialization, non-transactional requests receive +// uncertainty intervals from their target leaseholder, using a clock reading +// from the leaseholder's local HLC as the local limit and this clock reading + +// the cluster's maximum clock skew as the global limit. +// +// It is somewhat non-intuitive that non-transactional requests need uncertainty +// intervals — after all, they receive their timestamp to 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 +// interval: +// +// 1. the request timestamp is allocated before consulting the replica's lease. +// 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 +// the time of allocation. This is true despite invariants 1 & 2 from above, +// because the replica allocating the timestamp is not yet the leaseholder. +// +// In cases where the replica that assigned the non-transactional request's +// timestamp takes over as the leaseholder after the timestamp allocation, we +// expect minimumLocalLimitForLeaseholder to forward the local uncertainty +// limit above TimestampFromServerClock, to the lease start time. +// +// For example, consider the following series of events: +// - client A writes k = v1 +// - leaseholder writes v1 at ts = 100 +// - client A receives ack for write +// - client B wants to read k using a non-txn request +// - follower replica with slower clock receives non-txn request +// - follower replica assigns request ts = 95 +// - lease transferred to follower replica with lease start time = 101 +// - non-txn request must use 101 as limit of uncertainty interval to ensure +// that it observes k = v1 in uncertainty interval, performs a server-side +// retry, bumps its read timestamp, and returns k = v1. Failure to do so +// would be a stale read +// +// 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 +// leaseholder (and previous leaseholders) thus far. This clock reading is +// not not guaranteed to lead the commit timestamp of all of these writes, +// especially if they are committed remotely and resolved after the request +// has received its clock reading but before the request begins evaluating. +// +// As a result, the non-transactional request needs an uncertainty interval +// with a global uncertainty limit far enough in advance of the leaseholder's +// local HLC clock to ensure that it considers any value that was part of a +// transaction which could have committed before the request was received by +// 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 +// +// 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. +// +// For example, consider the following series of events: +// - client A begins a txn and is assigned provisional commit timestamp = 95 +// - client A's txn performs a Put(k, v1) +// - leaseholder serves Put(k, v1), lays down intent at written_timestamp = 95 +// - client A's txn performs a write elsewhere and hits a WriteTooOldError +// that bumps its provisional commit timestamp to 100 +// - client A's txn refreshes to ts = 100. This notably happens without +// involvement of the leaseholder that served the Put (this is at the heart +// of #36431), so that leaseholder's clock is not updated +// - client A's txn commits remotely and client A receives the acknowledgment +// - client B initiates non-txn read of k +// - leaseholder assigns read timestamp ts = 97 +// - asynchronous intent resolution resolves the txn's intent at k, moving v1 +// to ts = 100 in the process +// - non-txn request must use an uncertainty interval that extends past 100 +// to ensure that it observes k = v1 in uncertainty interval, performs a +// 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 + +// D8 ———————————————————————————————————————————————— +// // ComputeInterval // // Observed timestamps allow transactions to avoid uncertainty related restarts @@ -200,7 +312,7 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // 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 D7 = ComputeInterval +var D8 = ComputeInterval // Ignore unused warnings. -var _, _, _, _, _, _, _ = D1, D2, D3, D4, D5, D6, D7 +var _, _, _, _, _, _, _, _ = D1, D2, D3, D4, D5, D6, D7, D8 diff --git a/pkg/kv/kvserver/uncertainty/interval.go b/pkg/kv/kvserver/uncertainty/interval.go index 94856cbeb6de..be9de5d8e4f3 100644 --- a/pkg/kv/kvserver/uncertainty/interval.go +++ b/pkg/kv/kvserver/uncertainty/interval.go @@ -37,6 +37,12 @@ import "github.com/cockroachdb/cockroach/pkg/util/hlc" // global limit for the purposes of uncertainty, because observed timestamps do // not apply to values with synthetic timestamps. // +// Uncertainty intervals also apply to non-transactional requests that require +// strong consistency (single-key linearizability). These requests defer their +// timestamp allocation to the leaseholder of their (single) range. They then +// establish an uncertainty interval and perform any uncertainty restarts on the +// server. +// // NOTE: LocalLimit can be empty if no observed timestamp has been captured on // the local node. However, if it is set, it must be <= GlobalLimit. type Interval struct { diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index f8fa5f827f6c..cd3f571bf4e0 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -58,10 +58,10 @@ import ( // 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=[,]] +// 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=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] +// scan [t=] [ts=[,]] [resolve [status=]] k= [end=] [inconsistent] [tombstones] [reverse] [failOnMoreRecent] [localUncertaintyLimit=[,]] [globalUncertaintyLimit=[,]] [max=] [targetbytes=] [avoidExcess] [allowEmpty] // // merge [ts=[,]] k= v= [raw] // @@ -675,11 +675,15 @@ func cmdGet(e *evalCtx) error { if e.hasArg("failOnMoreRecent") { opts.FailOnMoreRecent = true } + opts.Uncertainty = uncertainty.Interval{ + GlobalLimit: e.getTsWithName(nil, "globalUncertaintyLimit"), + LocalLimit: hlc.ClockTimestamp(e.getTsWithName(nil, "localUncertaintyLimit")), + } if opts.Txn != nil { - opts.Uncertainty = uncertainty.Interval{ - GlobalLimit: txn.GlobalUncertaintyLimit, - LocalLimit: hlc.ClockTimestamp(e.getTsWithName(nil, "localUncertaintyLimit")), + if !opts.Uncertainty.GlobalLimit.IsEmpty() { + e.Fatalf("globalUncertaintyLimit arg incompatible with txn") } + opts.Uncertainty.GlobalLimit = txn.GlobalUncertaintyLimit } val, intent, err := MVCCGet(e.ctx, e.engine, key, ts, opts) // NB: the error is returned below. This ensures the test can @@ -777,11 +781,15 @@ func cmdScan(e *evalCtx) error { if e.hasArg("failOnMoreRecent") { opts.FailOnMoreRecent = true } + opts.Uncertainty = uncertainty.Interval{ + GlobalLimit: e.getTsWithName(nil, "globalUncertaintyLimit"), + LocalLimit: hlc.ClockTimestamp(e.getTsWithName(nil, "localUncertaintyLimit")), + } if opts.Txn != nil { - opts.Uncertainty = uncertainty.Interval{ - GlobalLimit: txn.GlobalUncertaintyLimit, - LocalLimit: hlc.ClockTimestamp(e.getTsWithName(nil, "localUncertaintyLimit")), + if !opts.Uncertainty.GlobalLimit.IsEmpty() { + e.Fatalf("globalUncertaintyLimit arg incompatible with txn") } + opts.Uncertainty.GlobalLimit = txn.GlobalUncertaintyLimit } if e.hasArg("max") { var n int diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 2a9904aecaa8..b9f714db37f3 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -391,13 +391,14 @@ func (p *pebbleMVCCScanner) init( p.txnEpoch = txn.Epoch p.txnSequence = txn.Sequence p.txnIgnoredSeqNums = txn.IgnoredSeqNums - - p.uncertainty = ui - // We must check uncertainty even if p.ts.Less(p.uncertainty.LocalLimit) - // because the local uncertainty limit cannot be applied to values with - // synthetic timestamps. - p.checkUncertainty = p.ts.Less(p.uncertainty.GlobalLimit) } + + p.uncertainty = ui + // We must check uncertainty even if p.ts >= local_uncertainty_limit + // because the local uncertainty limit cannot be applied to values with + // synthetic timestamps. We are only able to skip uncertainty checks if + // p.ts >= global_uncertainty_limit. + p.checkUncertainty = p.ts.Less(p.uncertainty.GlobalLimit) } // get iterates exactly once and adds one KV to the result set. 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 f3eefbb0265d..41f854262a02 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 @@ -2146,3 +2146,916 @@ 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/util/hlc/doc.go b/pkg/util/hlc/doc.go index 6cb3f6a63278..e5bc1a604e65 100644 --- a/pkg/util/hlc/doc.go +++ b/pkg/util/hlc/doc.go @@ -97,6 +97,9 @@ 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 @@ -120,25 +123,19 @@ TODO(nvanbenschoten): Update the above on written timestamps after #72121. 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. For more, - see pkg/kv/kvserver/uncertainty/doc.go. - - - Non-transactional requests (Raft + BatchRequest channels). Most KV operations - in CockroachDB are transactional and receive their read timestamps from their - gateway's HLC clock they are instantiated. They use an uncertainty interval - (see below) to avoid stale reads in the presence of clock skew. - - The KV API also exposes the option to elide the transaction for requests - targeting a single range (which trivially applies to all point requests). - These requests do not carry a predetermined read timestamp; instead, this - timestamp is chosen from the HLC upon arrival at the leaseholder for the - range. Since the HLC clock always leads the timestamp if any write served - on the range, this will not result in stale reads, despite not using an - uncertainty interval for such requests. - - TODO(nvanbenschoten): this mechanism is currently broken for future-time - writes. We either need to give non-transactional requests uncertainty - intervals or remove them. See https://github.com/cockroachdb/cockroach/issues/58459. + 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. + These requests do not collect observed timestamps directly, but they do + establish an uncertainty interval immediately upon receipt by their target + leaseholder, using a clock reading from the leaseholder's local HLC as the + local limit and this clock reading + the cluster's maximum clock skew as the + global limit. This limit can be used to make claims about values that could + not have been written yet at the time that the non-transaction request first + reached the leaseholder node. + + For more, see pkg/kv/kvserver/uncertainty/doc.go. - Transaction retry errors (BatchRequest and DistSQL channels). TODO(nvanbenschoten/andreimatei): is this a real case where passing a remote