From b40e44ba7780735d7165351667fd87485b37213f Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 3 Mar 2022 13:42:01 -0500 Subject: [PATCH] kv/storage: introduce and track local timestamps for MVCC versions Fixes #36431. Fixes #49360. Replaces #72121. 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 called the "local timestamp". The MVCC key's existing version timestamp dictates the key'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 the MVCC key encoding, 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. \### 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): - observe TPC-C performance -- top-line performance -- uncertainty retry rate -- commit-wait rate (should be zero) - 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/restore_data_processor.go | 5 +- .../backupccl/restore_data_processor_test.go | 1 + pkg/ccl/changefeedccl/kvfeed/scanner.go | 2 +- pkg/cmd/roachtest/tests/acceptance.go | 1 + pkg/kv/bulk/sst_batcher.go | 1 + .../batcheval/cmd_add_sstable_test.go | 3 +- pkg/kv/kvserver/below_raft_protos_test.go | 3 +- pkg/kv/kvserver/client_replica_test.go | 247 ++ pkg/kv/kvserver/client_test.go | 9 + pkg/kv/kvserver/consistency_queue_test.go | 2 + pkg/kv/kvserver/gc/gc_iterator_test.go | 2 +- pkg/kv/kvserver/replica_consistency.go | 6 +- pkg/kv/kvserver/replica_consistency_diff.go | 29 +- pkg/kv/kvserver/replica_test.go | 19 +- pkg/kv/kvserver/uncertainty/doc.go | 125 +- pkg/kv/kvserver/uncertainty/interval.go | 23 +- pkg/kv/kvserver/uncertainty/interval_test.go | 110 +- pkg/roachpb/data.go | 1 + pkg/roachpb/errors.go | 3 + pkg/roachpb/internal_raft.proto | 2 + pkg/sql/row/kv_fetcher.go | 2 +- pkg/storage/batch_test.go | 13 +- pkg/storage/bench_pebble_test.go | 3 +- pkg/storage/bench_test.go | 2 +- pkg/storage/engine_key.go | 30 +- pkg/storage/engine_key_test.go | 131 +- pkg/storage/enginepb/decode.go | 50 +- pkg/storage/enginepb/decode_test.go | 2 +- pkg/storage/enginepb/mvcc.go | 47 +- pkg/storage/enginepb/mvcc.proto | 9 +- pkg/storage/enginepb/mvcc_test.go | 19 +- pkg/storage/mvcc.go | 63 +- pkg/storage/mvcc_history_test.go | 22 +- pkg/storage/mvcc_incremental_iterator.go | 4 +- pkg/storage/mvcc_incremental_iterator_test.go | 10 +- pkg/storage/mvcc_key.go | 206 +- pkg/storage/mvcc_key_test.go | 377 +- pkg/storage/mvcc_stats_test.go | 4 +- pkg/storage/mvcc_test.go | 12 +- pkg/storage/pebble.go | 22 +- pkg/storage/pebble_iterator.go | 5 +- pkg/storage/pebble_mvcc_scanner.go | 8 +- pkg/storage/pebble_mvcc_scanner_test.go | 6 +- pkg/storage/pebble_test.go | 36 +- pkg/storage/sst.go | 4 +- .../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/util/hlc/doc.go | 29 +- 51 files changed, 5124 insertions(+), 721 deletions(-) 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/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index 9f609e7236d3..f014529bde1c 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -475,9 +475,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 c100546f54fc..902ae7e5a3ba 100644 --- a/pkg/ccl/backupccl/restore_data_processor_test.go +++ b/pkg/ccl/backupccl/restore_data_processor_test.go @@ -390,6 +390,7 @@ func runTestIngest(t *testing.T, init func(*cluster.Settings)) { for i := range kvs { if i < len(expectedKVs) { expectedKVs[i].Key.Timestamp = kvs[i].Key.Timestamp + expectedKVs[i].Key.LocalTimestamp = hlc.ClockTimestamp{} } } diff --git a/pkg/ccl/changefeedccl/kvfeed/scanner.go b/pkg/ccl/changefeedccl/kvfeed/scanner.go index 3385643d57dc..45a34bc8fad4 100644 --- a/pkg/ccl/changefeedccl/kvfeed/scanner.go +++ b/pkg/ccl/changefeedccl/kvfeed/scanner.go @@ -236,7 +236,7 @@ func slurpScanResponse( for len(br) > 0 { var kv roachpb.KeyValue var err error - kv.Key, kv.Value.Timestamp, kv.Value.RawBytes, br, err = enginepb.ScanDecodeKeyValue(br) + kv.Key, kv.Value.Timestamp, _, kv.Value.RawBytes, br, err = enginepb.ScanDecodeKeyValue(br) if err != nil { return errors.Wrapf(err, `decoding changes for %s`, span) } 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/bulk/sst_batcher.go b/pkg/kv/bulk/sst_batcher.go index 878436148145..85b939f6a89f 100644 --- a/pkg/kv/bulk/sst_batcher.go +++ b/pkg/kv/bulk/sst_batcher.go @@ -201,6 +201,7 @@ func (b *SSTBatcher) AddMVCCKey(ctx context.Context, key storage.MVCCKey, value b.batchTS = b.db.Clock().Now() } key.Timestamp = b.batchTS + key.LocalTimestamp = hlc.ClockTimestamp{} } // Update the range currently represented in this batch, as necessary. diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 7635e89a539e..c5142aeed3c2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -1466,7 +1466,8 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsClosedTS(t *testing.T) kvs, err := storage.Scan(store.Engine(), roachpb.Key("key"), roachpb.Key("key").Next(), 0) require.NoError(t, err) require.Len(t, kvs, 1) - require.Equal(t, storage.MVCCKey{Key: roachpb.Key("key"), Timestamp: writeTS}, kvs[0].Key) + expKey := storage.MVCCKey{Key: roachpb.Key("key"), Timestamp: writeTS}.Normalize() + require.Equal(t, expKey, kvs[0].Key) v, err := roachpb.Value{RawBytes: kvs[0].Value}.GetBytes() require.NoError(t, err) require.Equal(t, "sst", string(v)) diff --git a/pkg/kv/kvserver/below_raft_protos_test.go b/pkg/kv/kvserver/below_raft_protos_test.go index 02c5e8027863..cc1e7ef6ff75 100644 --- a/pkg/kv/kvserver/below_raft_protos_test.go +++ b/pkg/kv/kvserver/below_raft_protos_test.go @@ -64,6 +64,7 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{ m := enginepb.NewPopulatedMVCCMetadata(r, false) m.Txn = nil // never populated below Raft m.Timestamp.Synthetic = nil // never populated below Raft + m.LocalTimestamp = nil // never populated below Raft if m.MergeTimestamp != nil { m.MergeTimestamp.Synthetic = nil // never populated below Raft } @@ -71,7 +72,7 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{ return m }, emptySum: 7551962144604783939, - populatedSum: 6170112718709472849, + populatedSum: 12812489297533931627, }, reflect.TypeOf(&enginepb.RangeAppliedState{}): { populatedConstructor: func(r *rand.Rand) protoutil.Message { 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/consistency_queue_test.go b/pkg/kv/kvserver/consistency_queue_test.go index 951f071e7aee..6a5a68e3a643 100644 --- a/pkg/kv/kvserver/consistency_queue_test.go +++ b/pkg/kv/kvserver/consistency_queue_test.go @@ -279,6 +279,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { // mock this out for a consistent string below. diff[0].Timestamp = hlc.Timestamp{Logical: 987, WallTime: 123} + diff[0].LocalTimestamp = hlc.ClockTimestamp{Logical: 678, WallTime: 234} act := diff.String() @@ -286,6 +287,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { +++ follower +0.000000123,987 "e" + ts:1970-01-01 00:00:00.000000123 +0000 UTC ++ localTs:0.000000234,678 + value:"\x00\x00\x00\x00\x01T" + raw mvcc_key/value: 6500000000000000007b000003db0d 000000000154 ` diff --git a/pkg/kv/kvserver/gc/gc_iterator_test.go b/pkg/kv/kvserver/gc/gc_iterator_test.go index 2f4533436a1e..0a380e23725d 100644 --- a/pkg/kv/kvserver/gc/gc_iterator_test.go +++ b/pkg/kv/kvserver/gc/gc_iterator_test.go @@ -41,7 +41,7 @@ func TestGCIterator(t *testing.T) { Key: storage.MVCCKey{ Key: k, Timestamp: hlc.Timestamp{WallTime: ts * time.Nanosecond.Nanoseconds()}, - }, + }.Normalize(), Value: val, }, txn: txn, diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index 58fb8794839e..651c90f43ec4 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -607,7 +607,8 @@ func (*Replica) sha512( if snapshot != nil { // Add (a copy of) the kv pair into the debug message. kv := roachpb.RaftSnapshotData_KeyValue{ - Timestamp: unsafeKey.Timestamp, + Timestamp: unsafeKey.Timestamp, + LocalTimestamp: unsafeKey.LocalTimestamp, } alloc, kv.Key = alloc.Copy(unsafeKey.Key, 0) alloc, kv.Value = alloc.Copy(unsafeValue, 0) @@ -683,7 +684,8 @@ func (*Replica) sha512( if snapshot != nil { // Add LeaseAppliedState to the diff. kv := roachpb.RaftSnapshotData_KeyValue{ - Timestamp: hlc.Timestamp{}, + Timestamp: hlc.Timestamp{}, + LocalTimestamp: hlc.ClockTimestamp{}, } kv.Key = keys.RangeAppliedStateKey(desc.RangeID) var v roachpb.Value diff --git a/pkg/kv/kvserver/replica_consistency_diff.go b/pkg/kv/kvserver/replica_consistency_diff.go index 8d132fa31e37..ef0a6afafbe1 100644 --- a/pkg/kv/kvserver/replica_consistency_diff.go +++ b/pkg/kv/kvserver/replica_consistency_diff.go @@ -24,10 +24,23 @@ import ( type ReplicaSnapshotDiff struct { // LeaseHolder is set to true of this kv pair is only present on the lease // holder. - LeaseHolder bool - Key roachpb.Key - Timestamp hlc.Timestamp - Value []byte + LeaseHolder bool + Key roachpb.Key + Timestamp hlc.Timestamp + LocalTimestamp hlc.ClockTimestamp + Value []byte +} + +func makeReplicaSnapshotDiff( + leaseholder bool, kv roachpb.RaftSnapshotData_KeyValue, +) ReplicaSnapshotDiff { + return ReplicaSnapshotDiff{ + LeaseHolder: leaseholder, + Key: kv.Key, + Timestamp: kv.Timestamp, + LocalTimestamp: kv.LocalTimestamp, + Value: kv.Value, + } } // ReplicaSnapshotDiffSlice groups multiple ReplicaSnapshotDiff records and @@ -45,6 +58,7 @@ func (rsds ReplicaSnapshotDiffSlice) SafeFormat(buf redact.SafePrinter, _ rune) } const format = `%s%s %s %s ts:%s +%s localTs:%s %s value:%s %s raw mvcc_key/value: %x %x ` @@ -52,6 +66,7 @@ func (rsds ReplicaSnapshotDiffSlice) SafeFormat(buf redact.SafePrinter, _ rune) buf.Printf(format, prefix, d.Timestamp, d.Key, prefix, d.Timestamp.GoTime(), + prefix, d.LocalTimestamp, prefix, SprintMVCCKeyValue(storage.MVCCKeyValue{Key: mvccKey, Value: d.Value}, false /* printKey */), prefix, storage.EncodeMVCCKey(mvccKey), d.Value) } @@ -78,11 +93,11 @@ func diffRange(l, r *roachpb.RaftSnapshotData) ReplicaSnapshotDiffSlice { } addLeaseHolder := func() { - diff = append(diff, ReplicaSnapshotDiff{LeaseHolder: true, Key: e.Key, Timestamp: e.Timestamp, Value: e.Value}) + diff = append(diff, makeReplicaSnapshotDiff(true, e)) i++ } addReplica := func() { - diff = append(diff, ReplicaSnapshotDiff{LeaseHolder: false, Key: v.Key, Timestamp: v.Timestamp, Value: v.Value}) + diff = append(diff, makeReplicaSnapshotDiff(false, v)) j++ } @@ -122,7 +137,7 @@ func diffRange(l, r *roachpb.RaftSnapshotData) ReplicaSnapshotDiffSlice { } else { addReplica() } - } else if !bytes.Equal(e.Value, v.Value) { + } else if !bytes.Equal(e.Value, v.Value) || !e.LocalTimestamp.Equal(v.LocalTimestamp) { addLeaseHolder() addReplica() } else { diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 491e532574d4..d4e4d5aa2560 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -7684,7 +7684,7 @@ func TestDiffRange(t *testing.T) { // Construct the two snapshots. leaderSnapshot := &roachpb.RaftSnapshotData{ KV: []roachpb.RaftSnapshotData_KeyValue{ - {Key: []byte("a"), Timestamp: timestamp, Value: value}, + {Key: []byte("a"), Timestamp: timestamp, LocalTimestamp: hlc.ClockTimestamp{WallTime: 111}, Value: value}, {Key: []byte("abc"), Timestamp: timestamp, Value: value}, {Key: []byte("abcd"), Timestamp: timestamp, Value: value}, {Key: []byte("abcde"), Timestamp: timestamp, Value: value}, @@ -7694,6 +7694,9 @@ func TestDiffRange(t *testing.T) { {Key: []byte("abcdefg"), Timestamp: timestamp, Value: value}, {Key: []byte("abcdefg"), Timestamp: timestamp.Add(0, -1), Value: value}, {Key: []byte("abcdefgh"), Timestamp: timestamp, Value: value}, + {Key: []byte("local1"), Timestamp: timestamp, LocalTimestamp: hlc.ClockTimestamp{WallTime: 111}, Value: value}, + {Key: []byte("local2"), Timestamp: timestamp, LocalTimestamp: hlc.ClockTimestamp{WallTime: 222}, Value: value}, + {Key: []byte("local3"), Timestamp: timestamp, LocalTimestamp: hlc.ClockTimestamp{WallTime: 333}, Value: value}, {Key: []byte("x"), Timestamp: timestamp, Value: value}, {Key: []byte("y"), Timestamp: timestamp, Value: value}, // Both 'zeroleft' and 'zeroright' share the version at (1,1), but @@ -7720,6 +7723,9 @@ func TestDiffRange(t *testing.T) { {Key: []byte("abcdefg"), Timestamp: timestamp.Add(0, 1), Value: value}, {Key: []byte("abcdefg"), Timestamp: timestamp, Value: value}, {Key: []byte("abcdefgh"), Timestamp: timestamp, Value: value}, + {Key: []byte("local1"), Timestamp: timestamp, LocalTimestamp: hlc.ClockTimestamp{WallTime: 111}, Value: value}, + {Key: []byte("local2"), Timestamp: timestamp, LocalTimestamp: hlc.ClockTimestamp{WallTime: 444}, Value: value}, + {Key: []byte("local3"), Timestamp: timestamp, LocalTimestamp: hlc.ClockTimestamp{WallTime: 333}, Value: value}, {Key: []byte("x"), Timestamp: timestamp, Value: []byte("bar")}, {Key: []byte("z"), Timestamp: timestamp, Value: value}, {Key: []byte("zeroleft"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, Value: value}, @@ -7730,12 +7736,14 @@ func TestDiffRange(t *testing.T) { // The expected diff. eDiff := ReplicaSnapshotDiffSlice{ - {LeaseHolder: true, Key: []byte("a"), Timestamp: timestamp, Value: value}, + {LeaseHolder: true, Key: []byte("a"), Timestamp: timestamp, LocalTimestamp: hlc.ClockTimestamp{WallTime: 111}, Value: value}, {LeaseHolder: false, Key: []byte("ab"), Timestamp: timestamp, Value: value}, {LeaseHolder: true, Key: []byte("abcd"), Timestamp: timestamp, Value: value}, {LeaseHolder: false, Key: []byte("abcdef"), Timestamp: timestamp, Value: value}, {LeaseHolder: false, Key: []byte("abcdefg"), Timestamp: timestamp.Add(0, 1), Value: value}, {LeaseHolder: true, Key: []byte("abcdefg"), Timestamp: timestamp.Add(0, -1), Value: value}, + {LeaseHolder: true, Key: []byte("local2"), Timestamp: timestamp, LocalTimestamp: hlc.ClockTimestamp{WallTime: 222}, Value: value}, + {LeaseHolder: false, Key: []byte("local2"), Timestamp: timestamp, LocalTimestamp: hlc.ClockTimestamp{WallTime: 444}, Value: value}, {LeaseHolder: true, Key: []byte("x"), Timestamp: timestamp, Value: value}, {LeaseHolder: false, Key: []byte("x"), Timestamp: timestamp, Value: []byte("bar")}, {LeaseHolder: true, Key: []byte("y"), Timestamp: timestamp, Value: value}, @@ -7748,7 +7756,7 @@ func TestDiffRange(t *testing.T) { for i, e := range eDiff { v := diff[i] - if e.LeaseHolder != v.LeaseHolder || !bytes.Equal(e.Key, v.Key) || e.Timestamp != v.Timestamp || !bytes.Equal(e.Value, v.Value) { + if !reflect.DeepEqual(e, v) { t.Fatalf("diff varies at row %d, want %v and got %v\n\ngot:\n%s\nexpected:\n%s", i, e, v, diff, eDiff) } } @@ -7763,22 +7771,27 @@ func TestDiffRange(t *testing.T) { +++ follower -0.000001729,1 "a" - ts:1970-01-01 00:00:00.000001729 +0000 UTC +- localTs:0.000000111,0 - value:"foo" - raw mvcc_key/value: 610000000000000006c1000000010d 666f6f +0.000001729,1 "ab" + ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ localTs:0,0 + value:"foo" + raw mvcc_key/value: 61620000000000000006c1000000010d 666f6f -0.000001729,1 "abcd" - ts:1970-01-01 00:00:00.000001729 +0000 UTC +- localTs:0,0 - value:"foo" - raw mvcc_key/value: 616263640000000000000006c1000000010d 666f6f +0.000001729,1 "abcdef" + ts:1970-01-01 00:00:00.000001729 +0000 UTC ++ localTs:0,0 + value:"foo" + raw mvcc_key/value: 6162636465660000000000000006c1000000010d 666f6f +0,0 "foo" + ts:1970-01-01 00:00:00 +0000 UTC ++ localTs:0,0 + value:"foo" + raw mvcc_key/value: 666f6f00 666f6f ` diff --git a/pkg/kv/kvserver/uncertainty/doc.go b/pkg/kv/kvserver/uncertainty/doc.go index 81fc80dc6941..d54dc8849ab6 100644 --- a/pkg/kv/kvserver/uncertainty/doc.go +++ b/pkg/kv/kvserver/uncertainty/doc.go @@ -20,7 +20,51 @@ // 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 +// and an optional "local" clock 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. +// +// 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 a versioned key 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 D0 = /* storage.MVCCKey */ struct { + Key roachpb.Key + Timestamp hlc.Timestamp + LocalTimestamp hlc.ClockTimestamp +}{} // D1 ———————————————————————————————————————————————— // @@ -91,7 +135,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 +162,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 keys track their local timestamp separately from +// their version timestamp. This provides the flexibility to advance a key'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 +203,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,14 +235,15 @@ 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. +// still places a bound on the local timestamps associated with values in the +// range that may have been written before the transaction began. var D6 = roachpb.Transaction{}.ObservedTimestamps // D7 ———————————————————————————————————————————————— @@ -219,7 +273,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 +283,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 +307,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 +320,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,13 +350,6 @@ 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 @@ -324,4 +371,4 @@ var D7 = roachpb.Header{}.TimestampFromServerClock var D8 = ComputeInterval // Ignore unused warnings. -var _, _, _, _, _, _, _, _ = D1, D2, D3, D4, D5, D6, D7, D8 +var _, _, _, _, _, _, _, _, _ = D0, 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 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 e0ef80f70e67..aa5bfed94c7d 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -946,6 +946,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/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/roachpb/internal_raft.proto b/pkg/roachpb/internal_raft.proto index 5837411d43ff..bf0ca9bab656 100644 --- a/pkg/roachpb/internal_raft.proto +++ b/pkg/roachpb/internal_raft.proto @@ -41,6 +41,8 @@ message RaftSnapshotData { optional bytes key = 1; optional bytes value = 2; optional util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp local_timestamp = 4 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; } repeated KeyValue KV = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "KV"]; diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index 6e619022c987..542556736a97 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -184,7 +184,7 @@ func (f *KVFetcher) NextKV( var ts hlc.Timestamp switch mvccDecodeStrategy { case MVCCDecodingRequired: - key, ts, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValue(f.batchResponse) + key, ts, _, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValue(f.batchResponse) case MVCCDecodingNotRequired: key, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValueNoTS(f.batchResponse) } diff --git a/pkg/storage/batch_test.go b/pkg/storage/batch_test.go index 43f9cff57c39..bdf69fff26d8 100644 --- a/pkg/storage/batch_test.go +++ b/pkg/storage/batch_test.go @@ -1110,6 +1110,14 @@ func TestDecodeKey(t *testing.T) { {Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 1}}, {Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}}, {Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1, Synthetic: true}}, + {Key: roachpb.Key("foo"), LocalTimestamp: hlc.ClockTimestamp{WallTime: 89999}}, + {Key: roachpb.Key("foo"), LocalTimestamp: hlc.ClockTimestamp{WallTime: 99, Logical: 45}}, + {Key: roachpb.Key("foo"), Timestamp: hlc.Timestamp{WallTime: 89999}, LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}}, + {Key: roachpb.Key("foo"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45}, LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}}, + {Key: roachpb.Key("foo"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45, Synthetic: true}, LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}}, + {Key: roachpb.Key("foo"), Timestamp: hlc.Timestamp{WallTime: 89999}, LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}}, + {Key: roachpb.Key("foo"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45}, LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}}, + {Key: roachpb.Key("foo"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45, Synthetic: true}, LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}}, } for _, test := range tests { t.Run(test.String(), func(t *testing.T) { @@ -1137,8 +1145,9 @@ func TestDecodeKey(t *testing.T) { if err != nil { t.Fatalf("unexpected err: %+v", err) } - if !reflect.DeepEqual(test, decodedKey) { - t.Errorf("expected %+v got %+v", test, decodedKey) + expDecoded := test.Normalize() + if !reflect.DeepEqual(expDecoded, decodedKey) { + t.Errorf("expected %+v got %+v", expDecoded, decodedKey) } }) } diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index 34f294f02cbc..9032b2947c3c 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -389,7 +389,8 @@ func BenchmarkBatchBuilderPut(b *testing.B) { for j := i; j < end; j++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(j))) ts := hlc.Timestamp{WallTime: int64(j)} - require.NoError(b, batch.Set(EncodeMVCCKey(MVCCKey{key, ts}), value, nil /* WriteOptions */)) + enc := EncodeMVCCKey(MVCCKey{Key: key, Timestamp: ts}) + require.NoError(b, batch.Set(enc, value, nil /* WriteOptions */)) } batch.Reset() } diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index 541a98a99815..6a43d2764215 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -1577,7 +1577,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)) } diff --git a/pkg/storage/engine_key.go b/pkg/storage/engine_key.go index fcf3b89dbfc2..e8f2dde2121b 100644 --- a/pkg/storage/engine_key.go +++ b/pkg/storage/engine_key.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" ) @@ -44,11 +45,13 @@ type EngineKey struct { // referencing these lengths and fix it. // TODO(nvanbenschoten): unify these constants with those in mvcc_key.go. const ( - engineKeyNoVersion = 0 - engineKeyVersionWallTimeLen = 8 - engineKeyVersionWallAndLogicalTimeLen = 12 - engineKeyVersionWallLogicalAndSyntheticTimeLen = 13 - engineKeyVersionLockTableLen = 17 + engineKeyNoVersion = 0 + engineKeyVersionWallTimeLen = 8 + engineKeyVersionWallAndLogicalTimeLen = 12 + engineKeyVersionWallLogicalAndSyntheticTimeLen = 13 + engineKeyVersionLockTableLen = 17 + engineKeyVersionWallAndLogicalTimeLocalWallTimeLen = 20 + engineKeyVersionWallAndLogicalTimeLocalWallAndLogicalTimeLen = 24 ) // Format implements the fmt.Formatter interface @@ -137,7 +140,9 @@ func (k EngineKey) IsMVCCKey() bool { return l == engineKeyNoVersion || l == engineKeyVersionWallTimeLen || l == engineKeyVersionWallAndLogicalTimeLen || - l == engineKeyVersionWallLogicalAndSyntheticTimeLen + l == engineKeyVersionWallLogicalAndSyntheticTimeLen || + l == engineKeyVersionWallAndLogicalTimeLocalWallTimeLen || + l == engineKeyVersionWallAndLogicalTimeLocalWallAndLogicalTimeLen } // IsLockTableKey returns true if the key can be decoded as a LockTableKey. @@ -153,13 +158,26 @@ func (k EngineKey) ToMVCCKey() (MVCCKey, error) { // No-op. case engineKeyVersionWallTimeLen: key.Timestamp.WallTime = int64(binary.BigEndian.Uint64(k.Version[0:8])) + key.LocalTimestamp = hlc.ClockTimestamp(key.Timestamp) // see MVCCKey.Normalize case engineKeyVersionWallAndLogicalTimeLen: key.Timestamp.WallTime = int64(binary.BigEndian.Uint64(k.Version[0:8])) key.Timestamp.Logical = int32(binary.BigEndian.Uint32(k.Version[8:12])) + key.LocalTimestamp = hlc.ClockTimestamp(key.Timestamp) // see MVCCKey.Normalize case engineKeyVersionWallLogicalAndSyntheticTimeLen: key.Timestamp.WallTime = int64(binary.BigEndian.Uint64(k.Version[0:8])) key.Timestamp.Logical = int32(binary.BigEndian.Uint32(k.Version[8:12])) + // TODO(nvanbenschoten): In v23.1, remove the Synthetic flag. key.Timestamp.Synthetic = k.Version[12] != 0 + key.LocalTimestamp = hlc.MinClockTimestamp // see MVCCKey.Normalize + case engineKeyVersionWallAndLogicalTimeLocalWallTimeLen: + key.Timestamp.WallTime = int64(binary.BigEndian.Uint64(k.Version[0:8])) + key.Timestamp.Logical = int32(binary.BigEndian.Uint32(k.Version[8:12])) + key.LocalTimestamp.WallTime = int64(binary.BigEndian.Uint64(k.Version[12:20])) + case engineKeyVersionWallAndLogicalTimeLocalWallAndLogicalTimeLen: + key.Timestamp.WallTime = int64(binary.BigEndian.Uint64(k.Version[0:8])) + key.Timestamp.Logical = int32(binary.BigEndian.Uint32(k.Version[8:12])) + key.LocalTimestamp.WallTime = int64(binary.BigEndian.Uint64(k.Version[12:20])) + key.LocalTimestamp.Logical = int32(binary.BigEndian.Uint32(k.Version[20:24])) default: return MVCCKey{}, errors.Errorf("version is not an encoded timestamp %x", k.Version) } diff --git a/pkg/storage/engine_key_test.go b/pkg/storage/engine_key_test.go index 80703857e117..244631a53d97 100644 --- a/pkg/storage/engine_key_test.go +++ b/pkg/storage/engine_key_test.go @@ -89,10 +89,72 @@ func TestMVCCAndEngineKeyEncodeDecode(t *testing.T) { {key: MVCCKey{Key: roachpb.Key("glue"), Timestamp: hlc.Timestamp{WallTime: 89999}}}, {key: MVCCKey{Key: roachpb.Key("foo"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45}}}, {key: MVCCKey{Key: roachpb.Key("bar"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45, Synthetic: true}}}, + {key: MVCCKey{Key: roachpb.Key("glue"), LocalTimestamp: hlc.ClockTimestamp{WallTime: 89999}}}, + {key: MVCCKey{Key: roachpb.Key("foo"), LocalTimestamp: hlc.ClockTimestamp{WallTime: 99, Logical: 45}}}, + {key: MVCCKey{ + Key: roachpb.Key("glue"), + Timestamp: hlc.Timestamp{WallTime: 99}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45, Synthetic: true}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("glue"), + Timestamp: hlc.Timestamp{WallTime: 99}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45, Synthetic: true}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("glue"), + Timestamp: hlc.Timestamp{WallTime: 89999}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 89999, Logical: 45}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 89999, Logical: 45, Synthetic: true}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("glue"), + Timestamp: hlc.Timestamp{WallTime: 89999}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 89999, Logical: 45}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 89999, Logical: 45, Synthetic: true}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, } for _, test := range testCases { t.Run("", func(t *testing.T) { - encodedTS := encodeMVCCTimestamp(test.key.Timestamp) + encodedTS := encodeMVCCTimestamp(test.key.Timestamp, test.key.LocalTimestamp) eKey := EngineKey{Key: test.key.Key, Version: encodedTS} b1 := eKey.Encode() require.Equal(t, len(b1), eKey.EncodedLen()) @@ -110,9 +172,10 @@ func TestMVCCAndEngineKeyEncodeDecode(t *testing.T) { require.False(t, eKeyDecoded.IsLockTableKey()) require.True(t, eKeyDecoded.IsMVCCKey()) require.NoError(t, eKeyDecoded.Validate()) + expDecoded := test.key.Normalize() keyDecoded, err := eKeyDecoded.ToMVCCKey() require.NoError(t, err) - require.Equal(t, test.key, keyDecoded) + require.Equal(t, expDecoded, keyDecoded) keyPart, ok := GetKeyPartFromEngineKey(b2) require.True(t, ok) require.Equal(t, eKeyDecoded.Key, roachpb.Key(keyPart)) @@ -138,6 +201,68 @@ func TestEngineKeyValidate(t *testing.T) { {key: MVCCKey{Key: roachpb.Key("glue"), Timestamp: hlc.Timestamp{WallTime: 89999}}}, {key: MVCCKey{Key: roachpb.Key("foo"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45}}}, {key: MVCCKey{Key: roachpb.Key("bar"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45, Synthetic: true}}}, + {key: MVCCKey{Key: roachpb.Key("glue"), LocalTimestamp: hlc.ClockTimestamp{WallTime: 89999}}}, + {key: MVCCKey{Key: roachpb.Key("foo"), LocalTimestamp: hlc.ClockTimestamp{WallTime: 99, Logical: 45}}}, + {key: MVCCKey{ + Key: roachpb.Key("glue"), + Timestamp: hlc.Timestamp{WallTime: 99}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45, Synthetic: true}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("glue"), + Timestamp: hlc.Timestamp{WallTime: 99}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45, Synthetic: true}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("glue"), + Timestamp: hlc.Timestamp{WallTime: 89999}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 89999, Logical: 45}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 89999, Logical: 45, Synthetic: true}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("glue"), + Timestamp: hlc.Timestamp{WallTime: 89999}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 89999, Logical: 45}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, + {key: MVCCKey{ + Key: roachpb.Key("foo"), + Timestamp: hlc.Timestamp{WallTime: 89999, Logical: 45, Synthetic: true}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1234, Logical: 21}, + }}, // Valid LockTableKeys. { @@ -168,7 +293,7 @@ func TestEngineKeyValidate(t *testing.T) { case EngineKey: ek = k case MVCCKey: - ek = EngineKey{Key: k.Key, Version: encodeMVCCTimestamp(k.Timestamp)} + ek = EngineKey{Key: k.Key, Version: encodeMVCCTimestamp(k.Timestamp, k.LocalTimestamp)} case LockTableKey: key, _ := k.ToEngineKey(nil) ek = key diff --git a/pkg/storage/enginepb/decode.go b/pkg/storage/enginepb/decode.go index 8b876dd79587..8f812ad551b6 100644 --- a/pkg/storage/enginepb/decode.go +++ b/pkg/storage/enginepb/decode.go @@ -25,6 +25,8 @@ import ( // code out from abstract interfaces -- See #30114 and #30001. // SplitMVCCKey returns the key and timestamp components of an encoded MVCC key. +// For MVCC keys with a local timestamp, that is also included in the timestamp +// component. func SplitMVCCKey(mvccKey []byte) (key []byte, ts []byte, ok bool) { if len(mvccKey) == 0 { return nil, nil, false @@ -42,32 +44,48 @@ func SplitMVCCKey(mvccKey []byte) (key []byte, ts []byte, ok bool) { return key, ts, true } -// DecodeKey decodes an key/timestamp from its serialized representation. -func DecodeKey(encodedKey []byte) ([]byte, hlc.Timestamp, error) { +// DecodeKey decodes a key, MVCC timestamp, and local timestamp from its +// serialized representation. +func DecodeKey(encodedKey []byte) ([]byte, hlc.Timestamp, hlc.ClockTimestamp, error) { key, encodedTS, ok := SplitMVCCKey(encodedKey) if !ok { - return nil, hlc.Timestamp{}, errors.Errorf("invalid encoded mvcc key: %x", encodedKey) + return nil, hlc.Timestamp{}, hlc.ClockTimestamp{}, errors.Errorf( + "invalid encoded mvcc key: %x", encodedKey) } // NB: This logic is duplicated with storage.decodeMVCCTimestamp() to avoid the // overhead of an additional function call (~13%). var timestamp hlc.Timestamp + var localTimestamp hlc.ClockTimestamp switch len(encodedTS) { case 0: // No-op. case 8: timestamp.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) + localTimestamp = hlc.ClockTimestamp(timestamp) // see MVCCKey.Normalize case 12: timestamp.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) timestamp.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + localTimestamp = hlc.ClockTimestamp(timestamp) // see MVCCKey.Normalize case 13: timestamp.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) timestamp.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + // TODO(nvanbenschoten): In v23.1, remove the Synthetic flag. timestamp.Synthetic = encodedTS[12] != 0 + localTimestamp = hlc.MinClockTimestamp // see MVCCKey.Normalize + case 20: + timestamp.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) + timestamp.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + localTimestamp.WallTime = int64(binary.BigEndian.Uint64(encodedTS[12:20])) + case 24: + timestamp.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) + timestamp.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + localTimestamp.WallTime = int64(binary.BigEndian.Uint64(encodedTS[12:20])) + localTimestamp.Logical = int32(binary.BigEndian.Uint32(encodedTS[20:24])) default: - return nil, hlc.Timestamp{}, errors.Errorf( + return nil, hlc.Timestamp{}, hlc.ClockTimestamp{}, errors.Errorf( "invalid encoded mvcc key: %x bad timestamp %x", encodedKey, encodedTS) } - return key, timestamp, nil + return key, timestamp, localTimestamp, nil } // kvLenSize is the number of bytes in the length prefix for each key/value @@ -78,24 +96,32 @@ const kvLenSize = 8 // ScanDecodeKeyValue decodes a key/value pair from a binary stream, such as in // an MVCCScan "batch" (this is not the RocksDB batch repr format), returning -// the key/value, the timestamp, and the suffix of data remaining in the batch. +// the key/value, the version timestamp, the local timestamp, and the suffix of +// data remaining in the batch. func ScanDecodeKeyValue( repr []byte, -) (key []byte, ts hlc.Timestamp, value []byte, orepr []byte, err error) { +) ( + key []byte, + ts hlc.Timestamp, + localTs hlc.ClockTimestamp, + value []byte, + orepr []byte, + err error, +) { if len(repr) < kvLenSize { - return key, ts, nil, repr, errors.Errorf("unexpected batch EOF") + return key, ts, localTs, nil, repr, errors.Errorf("unexpected batch EOF") } valSize := binary.LittleEndian.Uint32(repr) keyEnd := binary.LittleEndian.Uint32(repr[4:kvLenSize]) + kvLenSize if (keyEnd + valSize) > uint32(len(repr)) { - return key, ts, nil, nil, errors.Errorf("expected %d bytes, but only %d remaining", - keyEnd+valSize, len(repr)) + return key, ts, localTs, nil, nil, errors.Errorf( + "expected %d bytes, but only %d remaining", keyEnd+valSize, len(repr)) } rawKey := repr[kvLenSize:keyEnd] value = repr[keyEnd : keyEnd+valSize] repr = repr[keyEnd+valSize:] - key, ts, err = DecodeKey(rawKey) - return key, ts, value, repr, err + key, ts, localTs, err = DecodeKey(rawKey) + return key, ts, localTs, value, repr, err } // ScanDecodeKeyValueNoTS decodes a key/value pair from a binary stream, such as diff --git a/pkg/storage/enginepb/decode_test.go b/pkg/storage/enginepb/decode_test.go index f1ae35de8b88..03d917367f45 100644 --- a/pkg/storage/enginepb/decode_test.go +++ b/pkg/storage/enginepb/decode_test.go @@ -32,7 +32,7 @@ func BenchmarkScanDecodeKeyValue(b *testing.B) { b.Run("getTs=true", func(b *testing.B) { for i := 0; i < b.N; i++ { var err error - _, _, _, _, err = enginepb.ScanDecodeKeyValue(rep) + _, _, _, _, _, err = enginepb.ScanDecodeKeyValue(rep) if err != nil { b.Fatal(err) } diff --git a/pkg/storage/enginepb/mvcc.go b/pkg/storage/enginepb/mvcc.go index 8e9c9a427633..ba273daaf8ee 100644 --- a/pkg/storage/enginepb/mvcc.go +++ b/pkg/storage/enginepb/mvcc.go @@ -14,6 +14,7 @@ import ( "math" "sort" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/redact" ) @@ -256,6 +257,38 @@ func (meta *MVCCMetadata) GetIntentValue(seq TxnSeq) ([]byte, bool) { return nil, false } +// GetLocalTimestamp returns the local timestamp of the most recent versioned +// value. If the local timestamp is not set explicitly, its implicit value is +// returned. +func (meta *MVCCMetadata) GetLocalTimestamp() hlc.ClockTimestamp { + if meta.LocalTimestamp == nil { + if meta.Timestamp.ToTimestamp().Synthetic { + return hlc.MinClockTimestamp + } + return hlc.ClockTimestamp(meta.Timestamp.ToTimestamp()) + } + return *meta.LocalTimestamp +} + +// SetLocalTimestamp sets the local timestamp field to the provided value, if +// necessary. See MVCCKey's LocalTimestampNeeded and Normalize methods for +// details on when the local timestamp can be omitted. +// +// The alloc field is optional and can be used to avoid a heap allocation in +// some cases. If supplied, the referenced memory must not be modified while +// the MVCCMetadata is in use. +func (meta *MVCCMetadata) SetLocalTimestamp(localTs hlc.ClockTimestamp, alloc *hlc.ClockTimestamp) { + if localTs.IsEmpty() || meta.Timestamp.ToTimestamp().LessEq(localTs.ToTimestamp()) { + meta.LocalTimestamp = nil + } else { + if alloc == nil { + alloc = new(hlc.ClockTimestamp) + } + *alloc = localTs + meta.LocalTimestamp = alloc + } +} + // String implements the fmt.Stringer interface. func (m MVCCMetadata_SequencedIntent) String() string { return redact.StringWithoutMarkers(m) @@ -278,13 +311,13 @@ func (meta *MVCCMetadata) String() string { func (meta *MVCCMetadata) SafeFormat(w redact.SafePrinter, _ rune) { expand := w.Flag('+') - w.Printf("txn={%s} ts=%s del=%t klen=%d vlen=%d", - meta.Txn, - meta.Timestamp, - meta.Deleted, - meta.KeyBytes, - meta.ValBytes, - ) + w.Printf("txn={%s} ts=%s", meta.Txn, meta.Timestamp) + + if meta.LocalTimestamp != nil { + w.Printf(" localTs=%s", meta.LocalTimestamp) + } + + w.Printf(" del=%t klen=%d vlen=%d", meta.Deleted, meta.KeyBytes, meta.ValBytes) if len(meta.RawBytes) > 0 { if expand { diff --git a/pkg/storage/enginepb/mvcc.proto b/pkg/storage/enginepb/mvcc.proto index ffaf911e898e..b7ce85a2698f 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -15,6 +15,7 @@ option go_package = "enginepb"; import "storage/enginepb/mvcc3.proto"; import "util/hlc/legacy_timestamp.proto"; +import "util/hlc/timestamp.proto"; import "gogoproto/gogo.proto"; // MVCCMetadata holds MVCC metadata for a key. Used by storage/mvcc.go. @@ -34,6 +35,13 @@ message MVCCMetadata { // only one version, the data is stored inline (via raw_bytes), and // timestamp is set to zero. optional util.hlc.LegacyTimestamp timestamp = 2 [(gogoproto.nullable) = false]; + // The local timestamp of the most recent versioned value if this is a + // value that may have multiple versions. For values which may have only + // one version, this timestamp is set to nil. + // NOTE: Code should use the GetLocalTimestamp/SetLocalTimestamp methods + // to access this field. + optional util.hlc.Timestamp local_timestamp = 10 [ + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; // Is the most recent value a deletion tombstone? optional bool deleted = 3 [(gogoproto.nullable) = false]; // The size in bytes of the most recent encoded key. @@ -66,7 +74,6 @@ message MVCCMetadata { // for the key along with each values corresponding Sequence. It doesn't // contain the latest intent value but rather stores all the values that have // been overwritten by the transaction. - // IntentHistory will be empty for non-transactional requests. repeated SequencedIntent intent_history = 8 [(gogoproto.nullable) = false]; // This provides a measure of protection against replays caused by // Raft duplicating merge commands. diff --git a/pkg/storage/enginepb/mvcc_test.go b/pkg/storage/enginepb/mvcc_test.go index 7acf323287ff..c168aadb98ad 100644 --- a/pkg/storage/enginepb/mvcc_test.go +++ b/pkg/storage/enginepb/mvcc_test.go @@ -27,6 +27,7 @@ func TestFormatMVCCMetadata(t *testing.T) { t.Fatal(err) } ts := hlc.Timestamp{Logical: 1} + localTs := hlc.ClockTimestamp{Logical: 2} txnDidNotUpdateMeta := true tmeta := &enginepb.TxnMeta{ Key: roachpb.Key("a"), @@ -43,11 +44,12 @@ func TestFormatMVCCMetadata(t *testing.T) { val3 := roachpb.Value{} val3.SetString("baz") meta := &enginepb.MVCCMetadata{ - Txn: tmeta, - Timestamp: ts.ToLegacyTimestamp(), - KeyBytes: 123, - ValBytes: 456, - RawBytes: val1.RawBytes, + Txn: tmeta, + Timestamp: ts.ToLegacyTimestamp(), + LocalTimestamp: &localTs, + KeyBytes: 123, + ValBytes: 456, + RawBytes: val1.RawBytes, IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ {Sequence: 11, Value: val2.RawBytes}, {Sequence: 22, Value: val3.RawBytes}, @@ -56,7 +58,8 @@ func TestFormatMVCCMetadata(t *testing.T) { } const expStr = `txn={id=d7aa0f5e key="a" pri=0.00000000 epo=1 ts=0,1 min=0,1 seq=0}` + - ` ts=0,1 del=false klen=123 vlen=456 rawlen=8 nih=2 mergeTs= txnDidNotUpdateMeta=true` + ` ts=0,1 localTs=0,2 del=false klen=123 vlen=456 rawlen=8 nih=2 mergeTs=` + + ` txnDidNotUpdateMeta=true` if str := meta.String(); str != expStr { t.Errorf( @@ -66,8 +69,8 @@ func TestFormatMVCCMetadata(t *testing.T) { } const expV = `txn={id=d7aa0f5e key=‹"a"› pri=0.00000000 epo=1 ts=0,1 min=0,1 seq=0}` + - ` ts=0,1 del=false klen=123 vlen=456 raw=‹/BYTES/foo› ih={{11 ‹/BYTES/bar›}{22 ‹/BYTES/baz›}}` + - ` mergeTs= txnDidNotUpdateMeta=true` + ` ts=0,1 localTs=0,2 del=false klen=123 vlen=456 raw=‹/BYTES/foo›` + + ` ih={{11 ‹/BYTES/bar›}{22 ‹/BYTES/baz›}} mergeTs= txnDidNotUpdateMeta=true` if str := redact.Sprintf("%+v", meta); str != expV { t.Errorf( diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index cddeca976c5b..76c7421460ca 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -849,7 +849,11 @@ func MVCCGetAsTxn( // iterator has already been seeked to metaKey, so a wasteful seek can be // avoided. func mvccGetMetadata( - iter MVCCIterator, metaKey MVCCKey, iterAlreadyPositioned bool, meta *enginepb.MVCCMetadata, + iter MVCCIterator, + metaKey MVCCKey, + iterAlreadyPositioned bool, + meta *enginepb.MVCCMetadata, + localTs *hlc.ClockTimestamp, ) (ok bool, keyBytes, valBytes int64, err error) { if iter == nil { return false, 0, 0, nil @@ -882,6 +886,7 @@ func mvccGetMetadata( meta.ValBytes = int64(len(iter.UnsafeValue())) meta.Deleted = meta.ValBytes == 0 meta.Timestamp = unsafeKey.Timestamp.ToLegacyTimestamp() + meta.SetLocalTimestamp(unsafeKey.LocalTimestamp, localTs) return true, int64(unsafeKey.EncodedSize()) - meta.KeyBytes, 0, nil } @@ -890,10 +895,12 @@ func mvccGetMetadata( // allocations. Managing this temporary buffer using a sync.Pool // completely eliminates allocation from the put common path. type putBuffer struct { - meta enginepb.MVCCMetadata - newMeta enginepb.MVCCMetadata - ts hlc.LegacyTimestamp - tmpbuf []byte + meta enginepb.MVCCMetadata + newMeta enginepb.MVCCMetadata + ts hlc.LegacyTimestamp // avoids heap allocations + localTs hlc.ClockTimestamp // avoids heap allocations + newLocalTs hlc.ClockTimestamp // avoids heap allocations + tmpbuf []byte // avoids heap allocations } var putBufferPool = sync.Pool{ @@ -1118,8 +1125,8 @@ func maybeGetValue( // "batch" (this is not the RocksDB batch repr format), returning both the // key/value and the suffix of data remaining in the batch. func MVCCScanDecodeKeyValue(repr []byte) (key MVCCKey, value []byte, orepr []byte, err error) { - k, ts, value, orepr, err := enginepb.ScanDecodeKeyValue(repr) - return MVCCKey{k, ts}, value, orepr, err + k, ts, localTs, value, orepr, err := enginepb.ScanDecodeKeyValue(repr) + return MVCCKey{k, ts, localTs}, value, orepr, err } // MVCCScanDecodeKeyValues decodes all key/value pairs returned in one or more @@ -1298,8 +1305,8 @@ func mvccPutInternal( } metaKey := MakeMVCCMetadataKey(key) - ok, origMetaKeySize, origMetaValSize, err := - mvccGetMetadata(iter, metaKey, false /* iterAlreadyPositioned */, &buf.meta) + ok, origMetaKeySize, origMetaValSize, err := mvccGetMetadata( + iter, metaKey, false /* iterAlreadyPositioned */, &buf.meta, &buf.localTs) if err != nil { return err } @@ -1466,13 +1473,16 @@ func mvccPutInternal( // delete the old intent, taking care with MVCC stats. logicalOp = MVCCUpdateIntentOpType if metaTimestamp.Less(writeTimestamp) { + versionKey := metaKey + versionKey.Timestamp = metaTimestamp + versionKey.LocalTimestamp = meta.GetLocalTimestamp() + { // 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 } @@ -1482,8 +1492,6 @@ func mvccPutInternal( iter = nil // prevent accidental use below } - versionKey := metaKey - versionKey.Timestamp = metaTimestamp if err := writer.ClearMVCC(versionKey); err != nil { return err } @@ -1589,6 +1597,7 @@ func mvccPutInternal( } buf.newMeta.Txn = txnMeta buf.newMeta.Timestamp = writeTimestamp.ToLegacyTimestamp() + buf.newMeta.SetLocalTimestamp(localTimestamp, &buf.newLocalTs) } newMeta := &buf.newMeta @@ -1630,7 +1639,8 @@ func mvccPutInternal( // RocksDB's skiplist memtable implementation includes a fast-path for // sequential insertion patterns. versionKey := metaKey - versionKey.Timestamp = writeTimestamp + versionKey.Timestamp = newMeta.Timestamp.ToTimestamp() + versionKey.LocalTimestamp = newMeta.GetLocalTimestamp() if err := writer.PutMVCC(versionKey, value); err != nil { return err } @@ -2010,6 +2020,7 @@ func MVCCClearTimeRange( if bufSize < useClearRangeThreshold { buf[bufSize].Key = append(buf[bufSize].Key[:0], k.Key...) buf[bufSize].Timestamp = k.Timestamp + buf[bufSize].LocalTimestamp = k.LocalTimestamp bufSize++ } else { // Buffer is now full -- switch to just tracking the start of the range @@ -2091,8 +2102,8 @@ func MVCCClearTimeRange( defer iter.Close() var clearedMetaKey MVCCKey - var clearedMeta enginepb.MVCCMetadata - var restoredMeta enginepb.MVCCMetadata + var clearedMeta, restoredMeta enginepb.MVCCMetadata + var clearedLocalTs, restoredLocalTs hlc.ClockTimestamp iter.SeekGE(MVCCKey{Key: key}) for { if ok, err := iter.Valid(); err != nil { @@ -2113,6 +2124,7 @@ func MVCCClearTimeRange( restoredMeta.Deleted = valueSize == 0 restoredMeta.ValBytes = valueSize restoredMeta.Timestamp = k.Timestamp.ToLegacyTimestamp() + restoredMeta.SetLocalTimestamp(k.LocalTimestamp, &restoredLocalTs) ms.Add(updateStatsOnClear( clearedMetaKey.Key, metaKeySize, 0, metaKeySize, 0, &clearedMeta, &restoredMeta, k.Timestamp.WallTime, @@ -2139,6 +2151,7 @@ func MVCCClearTimeRange( clearedMeta.ValBytes = int64(len(iter.UnsafeValue())) clearedMeta.Deleted = clearedMeta.ValBytes == 0 clearedMeta.Timestamp = k.Timestamp.ToLegacyTimestamp() + clearedMeta.SetLocalTimestamp(k.LocalTimestamp, &clearedLocalTs) // Move the iterator to the next key/value in linear iteration even if it // lies outside (startTime, endTime]. @@ -2996,7 +3009,7 @@ func mvccResolveWriteIntent( // testing. inProgress := !intent.Status.IsFinalized() && meta.Txn.Epoch >= intent.Txn.Epoch pushed := inProgress && timestampChanged - latestKey := MVCCKey{Key: intent.Key, Timestamp: metaTimestamp} + latestKey := MVCCKey{Key: intent.Key, Timestamp: metaTimestamp, LocalTimestamp: meta.GetLocalTimestamp()} // Handle partial txn rollbacks. If the current txn sequence // is part of a rolled back (ignored) seqnum range, we're going @@ -3067,6 +3080,13 @@ func mvccResolveWriteIntent( // Set the timestamp for upcoming write (or at least the stats update). buf.newMeta.Timestamp = newTimestamp.ToLegacyTimestamp() buf.newMeta.Txn.WriteTimestamp = newTimestamp + // 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. + localTs := latestKey.LocalTimestamp + localTs.Forward(intent.ClockWhilePending.Timestamp) + buf.newMeta.SetLocalTimestamp(localTs, &buf.newLocalTs) // Update or remove the metadata key. var metaKeySize, metaValSize int64 @@ -3090,8 +3110,10 @@ func mvccResolveWriteIntent( // rewrite it. 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 = buf.newMeta.Timestamp.ToTimestamp() + newKey.LocalTimestamp = buf.newMeta.GetLocalTimestamp() // Rewrite the versioned value at the new timestamp. iter.SeekGE(oldKey) @@ -3512,10 +3534,11 @@ func MVCCGarbageCollect( // Iterate through specified GC keys. meta := &enginepb.MVCCMetadata{} + localTs := &hlc.ClockTimestamp{} for _, gcKey := range keys { encKey := MakeMVCCMetadataKey(gcKey.Key) ok, metaKeySize, metaValSize, err := - mvccGetMetadata(iter, encKey, false /* iterAlreadyPositioned */, meta) + mvccGetMetadata(iter, encKey, false /* iterAlreadyPositioned */, meta, localTs) if err != nil { return err } diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 702040da32b2..287b2056ec58 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=] @@ -543,14 +543,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 +614,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 +631,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 +664,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 +732,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 }) @@ -763,7 +769,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.go b/pkg/storage/mvcc_incremental_iterator.go index db4a5e397073..f50e39d9ec82 100644 --- a/pkg/storage/mvcc_incremental_iterator.go +++ b/pkg/storage/mvcc_incremental_iterator.go @@ -83,7 +83,8 @@ type MVCCIncrementalIterator struct { // For allocation avoidance, meta is used to store the timestamp of keys // regardless if they are metakeys. - meta enginepb.MVCCMetadata + meta enginepb.MVCCMetadata + localTs hlc.ClockTimestamp // avoids heap allocation // Configuration passed in MVCCIncrementalIterOptions. intentPolicy MVCCIncrementalIterIntentPolicy @@ -335,6 +336,7 @@ func (i *MVCCIncrementalIterator) initMetaAndCheckForIntentOrInlineError() error // The key is an MVCC value and not an intent or inline. i.meta.Reset() i.meta.Timestamp = unsafeKey.Timestamp.ToLegacyTimestamp() + i.meta.SetLocalTimestamp(unsafeKey.LocalTimestamp, &i.localTs) return nil } diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index ed086c2a57d9..4056c68a7f02 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -40,7 +40,7 @@ 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} + return MVCCKeyValue{Key: MVCCKey{Key: key, Timestamp: ts}.Normalize(), Value: value} } func makeKVTxn( @@ -1372,14 +1372,14 @@ func TestMVCCIncrementalIteratorIntentDeletion(t *testing.T) { kvs, err := slurpKVsInTimeRange(db, kA, ts0, ts1) require.NoError(t, err) require.Equal(t, []MVCCKeyValue{ - {Key: MVCCKey{Key: kA, Timestamp: ts1}, Value: vA1.RawBytes}, + {Key: MVCCKey{Key: kA, Timestamp: ts1, LocalTimestamp: hlc.ClockTimestamp(ts1)}, Value: vA1.RawBytes}, }, kvs) // kA has a value at ts2. Again the intent is too new (ts3), so ignore. kvs, err = slurpKVsInTimeRange(db, kA, ts0, ts2) require.NoError(t, err) require.Equal(t, []MVCCKeyValue{ - {Key: MVCCKey{Key: kA, Timestamp: ts2}, Value: vA2.RawBytes}, - {Key: MVCCKey{Key: kA, Timestamp: ts1}, Value: vA1.RawBytes}, + {Key: MVCCKey{Key: kA, Timestamp: ts2, LocalTimestamp: hlc.ClockTimestamp(ts2)}, Value: vA2.RawBytes}, + {Key: MVCCKey{Key: kA, Timestamp: ts1, LocalTimestamp: hlc.ClockTimestamp(ts1)}, Value: vA1.RawBytes}, }, kvs) // At ts3, we should see the new intent _, err = slurpKVsInTimeRange(db, kA, ts0, ts3) @@ -1391,7 +1391,7 @@ func TestMVCCIncrementalIteratorIntentDeletion(t *testing.T) { kvs, err = slurpKVsInTimeRange(db, kB, ts0, ts1) require.NoError(t, err) require.Equal(t, []MVCCKeyValue{ - {Key: MVCCKey{Key: kB, Timestamp: ts1}, Value: vB1.RawBytes}, + {Key: MVCCKey{Key: kB, Timestamp: ts1, LocalTimestamp: hlc.ClockTimestamp(ts1)}, Value: vB1.RawBytes}, }, kvs) // Sanity check that we see the still unresolved intent for kC ts1. diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 40d75745af12..617ad0555477 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -36,10 +36,40 @@ const ( ) // MVCCKey is a versioned key, distinguished from roachpb.Key with the addition -// of a timestamp. +// of a "version" timestamp and an optional "local" clock 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. +// +// 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 pkg/kv/kvserver/observedts/doc.go. +// +// If the local timestamp is empty, it is assumed to be equal to the 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. type MVCCKey struct { - Key roachpb.Key - Timestamp hlc.Timestamp + Key roachpb.Key + Timestamp hlc.Timestamp + LocalTimestamp hlc.ClockTimestamp } // MakeMVCCMetadataKey creates an MVCCKey from a roachpb.Key. @@ -48,6 +78,7 @@ func MakeMVCCMetadataKey(key roachpb.Key) MVCCKey { } // Next returns the next key. +// The returned key will never contain a LocalTimestamp. func (k MVCCKey) Next() MVCCKey { ts := k.Timestamp.Prev() if ts.IsEmpty() { @@ -92,6 +123,7 @@ func (k MVCCKey) Less(l MVCCKey) bool { } // Equal returns whether two keys are identical. +// The method does not consider either key's LocalTimestamp. func (k MVCCKey) Equal(l MVCCKey) bool { return k.Key.Compare(l.Key) == 0 && k.Timestamp.EqOrdering(l.Timestamp) } @@ -108,7 +140,7 @@ func (k MVCCKey) IsValue() bool { func (k MVCCKey) EncodedSize() int { n := len(k.Key) + 1 if k.IsValue() { - // Note that this isn't quite accurate: timestamps consume between 8-13 + // Note that this isn't quite accurate: timestamps consume between 9-25 // bytes. Fixing this only adjusts the accounting for timestamps, not the // actual on disk storage. n += int(MVCCVersionTimestampSize) @@ -121,12 +153,23 @@ func (k MVCCKey) String() string { if !k.IsValue() { return k.Key.String() } - return fmt.Sprintf("%s/%s", k.Key, k.Timestamp) + if k.Timestamp.Synthetic || !k.LocalTimestampNeeded() { + return fmt.Sprintf("%s/%s", k.Key, k.Timestamp) + } + return fmt.Sprintf("%s/%s@%s", k.Key, k.Timestamp, k.LocalTimestamp) } // Format implements the fmt.Formatter interface. func (k MVCCKey) Format(f fmt.State, c rune) { - fmt.Fprintf(f, "%s/%s", k.Key, k.Timestamp) + // TODO(nvanbenschoten): add the following logic to mirror String. + // if !k.IsValue() { + // fmt.Fprintf(f, "%s", k.Key) + // } else if { ... + if k.Timestamp.Synthetic || !k.LocalTimestampNeeded() { + fmt.Fprintf(f, "%s/%s", k.Key, k.Timestamp) + } else { + fmt.Fprintf(f, "%s/%s@%s", k.Key, k.Timestamp, k.LocalTimestamp) + } } // Len returns the size of the MVCCKey when encoded. Implements the @@ -135,21 +178,52 @@ func (k MVCCKey) Len() int { return encodedMVCCKeyLength(k) } +// LocalTimestampNeeded returns whether the MVCCKey's local timestamp is needed, +// or whether it can be implied by (i.e. set to the same value as) its version +// timestamp. +func (k MVCCKey) LocalTimestampNeeded() bool { + // If the local timestamp is empty, it is assumed to be equal to the version + // timestamp and so the local timestamp is not needed. + return !k.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. + k.LocalTimestamp.ToTimestamp().Less(k.Timestamp) +} + +// Normalize returns a normalized version of the MVCCKey. The returned MVCCKey +// is equivalent to the result of encodings and then decoding the receiver. +func (k MVCCKey) Normalize() MVCCKey { + if k.Timestamp.Synthetic { + k.LocalTimestamp = hlc.MinClockTimestamp + } else if !k.LocalTimestampNeeded() { + k.LocalTimestamp = hlc.ClockTimestamp(k.Timestamp) + } + return k +} + // EncodeMVCCKey encodes an MVCCKey into its Pebble representation. The encoding // takes the following forms, where trailing time components are omitted when // zero-valued: // +// [key] [sentinel] [timeWall] [timeLogical] [localTimeWall] [localTimeLogical] [timeLength] +// [key] [sentinel] [timeWall] [timeLogical] [localTimeWall] [timeLength] // [key] [sentinel] [timeWall] [timeLogical] [timeSynthetic] [timeLength] // [key] [sentinel] [timeWall] [timeLogical] [timeLength] // [key] [sentinel] [timeWall] [timeLength] // [key] [sentinel] // -// key: the unmodified binary key (variable length) -// sentinel: separates key and timestamp (1 byte: 0x00) -// timeWall: Timestamp.WallTime (8 bytes: big-endian uint64) -// timeLogical: Timestamp.Logical (4 bytes: big-endian uint32) -// timeSynthetic: Timestamp.Synthetic (1 byte: 0x01 when set) -// timeLength: encoded timestamp length inc. itself (1 byte: uint8) +// key: the unmodified binary key (variable length) +// sentinel: separates key and timestamp (1 byte: 0x00) +// timeWall: Timestamp.WallTime (8 bytes: big-endian uint64) +// timeLogical: Timestamp.Logical (4 bytes: big-endian uint32) +// timeSynthetic: Timestamp.Synthetic (1 byte: 0x01 when set) +// localTimeWall: LocalTimestamp.WallTime (8 bytes: big-endian uint64) +// localTimeLogical: LocalTimestamp.Logical (4 bytes: big-endian uint32) +// timeLength: encoded timestamp length inc. itself (1 byte: uint8) // // The sentinel byte can be used to detect a key without a timestamp, since // timeLength will never be 0 (it includes itself in the length). @@ -190,47 +264,58 @@ func encodeMVCCKeyToBuf(buf []byte, key MVCCKey, keyLen int) { tsLen := keyLen - pos - mvccEncodedTimeLengthLen if tsLen > 0 { - encodeMVCCTimestampToBuf(buf[pos:], key.Timestamp) + encodeMVCCTimestampToBuf(buf[pos:], key.Timestamp, key.LocalTimestamp) pos += tsLen buf[pos] = byte(tsLen + mvccEncodedTimeLengthLen) } } -// encodeMVCCTimestamp encodes an MVCC timestamp into its Pebble -// representation, excluding length suffix and sentinel byte. -func encodeMVCCTimestamp(ts hlc.Timestamp) []byte { - tsLen := encodedMVCCTimestampLength(ts) +// encodeMVCCTimestamp encodes an MVCC timestamp and a local timestamp into +// their Pebble representation, excluding length suffix and sentinel byte. +func encodeMVCCTimestamp(ts hlc.Timestamp, localTs hlc.ClockTimestamp) []byte { + tsLen := encodedMVCCTimestampLength(ts, localTs) if tsLen == 0 { return nil } buf := make([]byte, tsLen) - encodeMVCCTimestampToBuf(buf, ts) + encodeMVCCTimestampToBuf(buf, ts, localTs) return buf } -// EncodeMVCCTimestampSuffix encodes an MVCC timestamp into its Pebble -// representation, including the length suffix but excluding the sentinel byte. -// This is equivalent to the Pebble suffix. -func EncodeMVCCTimestampSuffix(ts hlc.Timestamp) []byte { - tsLen := encodedMVCCTimestampLength(ts) +// EncodeMVCCTimestampSuffix encodes an MVCC timestamp and a local timestamp +// into their Pebble representation, including the length suffix but excluding +// the sentinel byte. This is equivalent to the Pebble suffix. +func EncodeMVCCTimestampSuffix(ts hlc.Timestamp, localTs hlc.ClockTimestamp) []byte { + tsLen := encodedMVCCTimestampLength(ts, localTs) if tsLen == 0 { return nil } buf := make([]byte, tsLen+mvccEncodedTimeLengthLen) - encodeMVCCTimestampToBuf(buf, ts) + encodeMVCCTimestampToBuf(buf, ts, localTs) buf[tsLen] = byte(tsLen + mvccEncodedTimeLengthLen) return buf } -// encodeMVCCTimestampToBuf encodes an MVCC timestamp into its Pebble -// representation, excluding the length suffix and sentinel byte. The target -// buffer must have the correct size, and the timestamp must not be empty. -func encodeMVCCTimestampToBuf(buf []byte, ts hlc.Timestamp) { +// encodeMVCCTimestampToBuf encodes an MVCC timestamp and a local timestamp into +// their Pebble representation, excluding the length suffix and sentinel byte. +// The target buffer must have the correct size, and the timestamp must not be +// empty. +func encodeMVCCTimestampToBuf(buf []byte, ts hlc.Timestamp, localTs hlc.ClockTimestamp) { binary.BigEndian.PutUint64(buf, uint64(ts.WallTime)) - if ts.Logical != 0 || ts.Synthetic { - binary.BigEndian.PutUint32(buf[mvccEncodedTimeWallLen:], uint32(ts.Logical)) + const afterWallTime = mvccEncodedTimeWallLen + needLocalTs := !localTs.IsEmpty() && localTs.ToTimestamp().Less(ts) // LocalTimestampNeeded + if ts.Logical != 0 || ts.Synthetic || needLocalTs { + binary.BigEndian.PutUint32(buf[afterWallTime:], uint32(ts.Logical)) + const afterLogical = afterWallTime + mvccEncodedTimeLogicalLen if ts.Synthetic { - buf[mvccEncodedTimeWallLen+mvccEncodedTimeLogicalLen] = 1 + // TODO(nvanbenschoten): stop writing Synthetic timestamps in v23.1. + buf[afterLogical] = 1 + } else if needLocalTs { + binary.BigEndian.PutUint64(buf[afterLogical:], uint64(localTs.WallTime)) + const afterLocalWallTime = afterLogical + mvccEncodedTimeWallLen + if localTs.Logical != 0 { + binary.BigEndian.PutUint32(buf[afterLocalWallTime:], uint32(localTs.Logical)) + } } } } @@ -243,10 +328,17 @@ func encodedMVCCKeyLength(key MVCCKey) int { keyLen := len(key.Key) + mvccEncodedTimeSentinelLen if !key.Timestamp.IsEmpty() { keyLen += mvccEncodedTimeWallLen + mvccEncodedTimeLengthLen - if key.Timestamp.Logical != 0 || key.Timestamp.Synthetic { + needLocalTs := key.LocalTimestampNeeded() + if key.Timestamp.Logical != 0 || key.Timestamp.Synthetic || needLocalTs { keyLen += mvccEncodedTimeLogicalLen if key.Timestamp.Synthetic { + // TODO(nvanbenschoten): stop writing Synthetic timestamps in v23.1. keyLen += mvccEncodedTimeSyntheticLen + } else if needLocalTs { + keyLen += mvccEncodedTimeWallLen + if key.LocalTimestamp.Logical != 0 { + keyLen += mvccEncodedTimeLogicalLen + } } } } @@ -260,13 +352,15 @@ func encodedMVCCKeyPrefixLength(key roachpb.Key) int { } // encodedMVCCTimestampLength returns the encoded length of the given MVCC -// timestamp, excluding the length suffix and sentinel bytes. -func encodedMVCCTimestampLength(ts hlc.Timestamp) int { +// timestamp and local timestamp, excluding the length suffix and sentinel +// bytes. +func encodedMVCCTimestampLength(ts hlc.Timestamp, localTs hlc.ClockTimestamp) int { // This is backwards, but encodedMVCCKeyLength() is called in the // EncodeMVCCKey() hot path and an additional function call to this function // shows ~6% overhead in benchmarks. We therefore do the timestamp length // calculation inline in encodedMVCCKeyLength(), and remove the excess here. - tsLen := encodedMVCCKeyLength(MVCCKey{Timestamp: ts}) - mvccEncodedTimeSentinelLen + key := MVCCKey{Timestamp: ts, LocalTimestamp: localTs} + tsLen := encodedMVCCKeyLength(key) - mvccEncodedTimeSentinelLen if tsLen > 0 { tsLen -= mvccEncodedTimeLengthLen } @@ -276,9 +370,9 @@ func encodedMVCCTimestampLength(ts hlc.Timestamp) int { // encodedMVCCTimestampSuffixLength returns the encoded length of the // given MVCC timestamp, including the length suffix. It returns 0 // if the timestamp is empty. -func encodedMVCCTimestampSuffixLength(ts hlc.Timestamp) int { +func encodedMVCCTimestampSuffixLength(ts hlc.Timestamp, localTs hlc.ClockTimestamp) int { // This is backwards, see comment in encodedMVCCTimestampLength() for why. - return encodedMVCCKeyLength(MVCCKey{Timestamp: ts}) - mvccEncodedTimeSentinelLen + return encodedMVCCKeyLength(MVCCKey{Timestamp: ts, LocalTimestamp: localTs}) - mvccEncodedTimeSentinelLen } // TODO(erikgrinaker): merge in the enginepb decoding functions once it can @@ -286,43 +380,57 @@ func encodedMVCCTimestampSuffixLength(ts hlc.Timestamp) int { // DecodeMVCCKey decodes an MVCCKey from its Pebble representation. func DecodeMVCCKey(encodedKey []byte) (MVCCKey, error) { - k, ts, err := enginepb.DecodeKey(encodedKey) - return MVCCKey{k, ts}, err + k, ts, localTs, err := enginepb.DecodeKey(encodedKey) + return MVCCKey{k, ts, localTs}, err } -// decodeMVCCTimestamp decodes an MVCC timestamp from its Pebble representation, -// excluding the length suffix. -func decodeMVCCTimestamp(encodedTS []byte) (hlc.Timestamp, error) { +// decodeMVCCTimestamp decodes an MVCC timestamp and local timestamp from their +// Pebble representation, excluding the length suffix. +func decodeMVCCTimestamp(encodedTS []byte) (hlc.Timestamp, hlc.ClockTimestamp, error) { // NB: This logic is duplicated in enginepb.DecodeKey() to avoid the // overhead of an additional function call there (~13%). var ts hlc.Timestamp + var localTs hlc.ClockTimestamp switch len(encodedTS) { case 0: // No-op. case 8: ts.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) + localTs = hlc.ClockTimestamp(ts) // see MVCCKey.Normalize case 12: ts.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) ts.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + localTs = hlc.ClockTimestamp(ts) // see MVCCKey.Normalize case 13: ts.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) ts.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + // TODO(nvanbenschoten): In v23.1, remove the Synthetic flag. ts.Synthetic = encodedTS[12] != 0 + localTs = hlc.MinClockTimestamp // see MVCCKey.Normalize + case 20: + ts.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) + ts.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + localTs.WallTime = int64(binary.BigEndian.Uint64(encodedTS[12:20])) + case 24: + ts.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) + ts.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + localTs.WallTime = int64(binary.BigEndian.Uint64(encodedTS[12:20])) + localTs.Logical = int32(binary.BigEndian.Uint32(encodedTS[20:24])) default: - return hlc.Timestamp{}, errors.Errorf("bad timestamp %x", encodedTS) + return hlc.Timestamp{}, hlc.ClockTimestamp{}, errors.Errorf("bad timestamp %x", encodedTS) } - return ts, nil + return ts, localTs, nil } -// decodeMVCCTimestampSuffix decodes an MVCC timestamp from its Pebble representation, -// including the length suffix. -func decodeMVCCTimestampSuffix(encodedTS []byte) (hlc.Timestamp, error) { +// decodeMVCCTimestampSuffix decodes an MVCC timestamp and a local timestamp +// from their Pebble representation, including the length suffix. +func decodeMVCCTimestampSuffix(encodedTS []byte) (hlc.Timestamp, hlc.ClockTimestamp, error) { if len(encodedTS) == 0 { - return hlc.Timestamp{}, nil + return hlc.Timestamp{}, hlc.ClockTimestamp{}, nil } encodedLen := len(encodedTS) if suffixLen := int(encodedTS[encodedLen-1]); suffixLen != encodedLen { - return hlc.Timestamp{}, errors.Errorf( + return hlc.Timestamp{}, hlc.ClockTimestamp{}, errors.Errorf( "bad timestamp: found length suffix %d, actual length %d", suffixLen, encodedLen) } return decodeMVCCTimestamp(encodedTS[:encodedLen-1]) diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index 4e8a2ea3d5b0..73c9f567bea4 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -68,12 +68,13 @@ func TestMVCCKeys(t *testing.T) { func TestMVCCKeyCompare(t *testing.T) { defer leaktest.AfterTest(t)() - a1 := MVCCKey{roachpb.Key("a"), hlc.Timestamp{Logical: 1}} - a2 := MVCCKey{roachpb.Key("a"), hlc.Timestamp{Logical: 2}} - b0 := MVCCKey{roachpb.Key("b"), hlc.Timestamp{Logical: 0}} - b1 := MVCCKey{roachpb.Key("b"), hlc.Timestamp{Logical: 1}} - b2 := MVCCKey{roachpb.Key("b"), hlc.Timestamp{Logical: 2}} - b2S := MVCCKey{roachpb.Key("b"), hlc.Timestamp{Logical: 2, Synthetic: true}} + a1 := MVCCKey{Key: roachpb.Key("a"), Timestamp: hlc.Timestamp{Logical: 1}} + a2 := MVCCKey{Key: roachpb.Key("a"), Timestamp: hlc.Timestamp{Logical: 2}} + b0 := MVCCKey{Key: roachpb.Key("b"), Timestamp: hlc.Timestamp{Logical: 0}} + b1 := MVCCKey{Key: roachpb.Key("b"), Timestamp: hlc.Timestamp{Logical: 1}} + b2 := MVCCKey{Key: roachpb.Key("b"), Timestamp: hlc.Timestamp{Logical: 2}} + b2S := MVCCKey{Key: roachpb.Key("b"), Timestamp: hlc.Timestamp{Logical: 2, Synthetic: true}} + b2L := MVCCKey{Key: roachpb.Key("b"), Timestamp: hlc.Timestamp{Logical: 2}, LocalTimestamp: hlc.ClockTimestamp{Logical: 1}} testcases := map[string]struct { a MVCCKey @@ -89,6 +90,7 @@ func TestMVCCKeyCompare(t *testing.T) { "set time gt empty": {b1, b0, 1}, // empty MVCC timestamps sort before non-empty "key time precedence": {a1, b2, -1}, // a before b, but 2 before 1; key takes precedence "synthetic equal": {b2, b2S, 0}, // synthetic bit does not affect ordering + "local equal": {b2, b2L, 0}, // local timestamp does not affect ordering } for name, tc := range testcases { t.Run(name, func(t *testing.T) { @@ -155,27 +157,208 @@ func (k randMVCCKey) Generate(r *rand.Rand, size int) reflect.Value { // NB: the zero timestamp cannot be synthetic. k.Timestamp.Synthetic = r.Intn(2) != 0 } + if r.Intn(2) != 0 { + k.LocalTimestamp.WallTime = r.Int63n(5) + k.LocalTimestamp.Logical = r.Int31n(5) + } return reflect.ValueOf(k) } -func TestEncodeDecodeMVCCKeyAndTimestampWithLength(t *testing.T) { +func TestEncodeDecodeMVCCKeyAndTimestampsWithLength(t *testing.T) { defer leaktest.AfterTest(t)() testcases := map[string]struct { key string ts hlc.Timestamp + localTs hlc.ClockTimestamp encoded string // hexadecimal }{ - "empty": {"", hlc.Timestamp{}, "00"}, - "only key": {"foo", hlc.Timestamp{}, "666f6f00"}, - "no key": {"", hlc.Timestamp{WallTime: 1643550788737652545}, "0016cf10bc0505574109"}, - "walltime": {"foo", hlc.Timestamp{WallTime: 1643550788737652545}, "666f6f0016cf10bc0505574109"}, - "logical": {"foo", hlc.Timestamp{Logical: 65535}, "666f6f0000000000000000000000ffff0d"}, - "synthetic": {"foo", hlc.Timestamp{Synthetic: true}, "666f6f00000000000000000000000000010e"}, - "walltime and logical": {"foo", hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, "666f6f0016cf10bc050557410000ffff0d"}, - "walltime and synthetic": {"foo", hlc.Timestamp{WallTime: 1643550788737652545, Synthetic: true}, "666f6f0016cf10bc0505574100000000010e"}, - "logical and synthetic": {"foo", hlc.Timestamp{Logical: 65535, Synthetic: true}, "666f6f0000000000000000000000ffff010e"}, - "all": {"foo", hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535, Synthetic: true}, "666f6f0016cf10bc050557410000ffff010e"}, + "empty": { + key: "", + ts: hlc.Timestamp{}, + localTs: hlc.ClockTimestamp{}, + encoded: "00", + }, + "only key": { + key: "foo", + ts: hlc.Timestamp{}, + localTs: hlc.ClockTimestamp{}, + encoded: "666f6f00", + }, + "no key": { + key: "", + ts: hlc.Timestamp{WallTime: 1643550788737652545}, + localTs: hlc.ClockTimestamp{}, + encoded: "0016cf10bc0505574109", + }, + "walltime": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545}, + localTs: hlc.ClockTimestamp{}, + encoded: "666f6f0016cf10bc0505574109", + }, + "logical": { + key: "foo", + ts: hlc.Timestamp{Logical: 65535}, + localTs: hlc.ClockTimestamp{}, + encoded: "666f6f0000000000000000000000ffff0d", + }, + "synthetic": { + key: "foo", + ts: hlc.Timestamp{Synthetic: true}, + localTs: hlc.ClockTimestamp{}, + encoded: "666f6f00000000000000000000000000010e", + }, + "walltime and logical": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, + localTs: hlc.ClockTimestamp{}, + encoded: "666f6f0016cf10bc050557410000ffff0d", + }, + "walltime and synthetic": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545, Synthetic: true}, + localTs: hlc.ClockTimestamp{}, + encoded: "666f6f0016cf10bc0505574100000000010e", + }, + "logical and synthetic": { + key: "foo", + ts: hlc.Timestamp{Logical: 65535, Synthetic: true}, + localTs: hlc.ClockTimestamp{}, + encoded: "666f6f0000000000000000000000ffff010e", + }, + "walltime, logical, and synthetic": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535, Synthetic: true}, + localTs: hlc.ClockTimestamp{}, + encoded: "666f6f0016cf10bc050557410000ffff010e", + }, + "no timestamp, local walltime": { + key: "foo", + ts: hlc.Timestamp{}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652545}, + // Implicit local timestamp. + encoded: "666f6f00", + }, + "no timestamp, local logical": { + key: "foo", + ts: hlc.Timestamp{}, + localTs: hlc.ClockTimestamp{Logical: 65535}, + // Implicit local timestamp. + encoded: "666f6f00", + }, + "no timestamp, local walltime and logical": { + key: "foo", + ts: hlc.Timestamp{}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 65535}, + // Implicit local timestamp. + encoded: "666f6f00", + }, + "walltime, smaller local walltime": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652544}, + encoded: "666f6f0016cf10bc050557410000000016cf10bc0505574015", + }, + "walltime, larger local walltime": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652546}, + // Implicit local timestamp. + encoded: "666f6f0016cf10bc0505574109", + }, + "walltime, local logical": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545}, + localTs: hlc.ClockTimestamp{Logical: 65535}, + encoded: "666f6f0016cf10bc050557410000000000000000000000000000ffff19", + }, + "walltime, smaller local walltime and logical": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652544, Logical: 65535}, + encoded: "666f6f0016cf10bc050557410000000016cf10bc050557400000ffff19", + }, + "walltime, larger local walltime and logical": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 65535}, + // Implicit local timestamp. + encoded: "666f6f0016cf10bc0505574109", + }, + "logical, local walltime": { + key: "foo", + ts: hlc.Timestamp{Logical: 65535}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652545}, + // Implicit local timestamp. + encoded: "666f6f0000000000000000000000ffff0d", + }, + "logical, smaller local logical": { + key: "foo", + ts: hlc.Timestamp{Logical: 65535}, + localTs: hlc.ClockTimestamp{Logical: 65534}, + encoded: "666f6f0000000000000000000000ffff00000000000000000000fffe19", + }, + "logical, larger local logical": { + key: "foo", + ts: hlc.Timestamp{Logical: 65535}, + localTs: hlc.ClockTimestamp{Logical: 65536}, + // Implicit local timestamp. + encoded: "666f6f0000000000000000000000ffff0d", + }, + "logical, local walltime and logical": { + key: "foo", + ts: hlc.Timestamp{Logical: 65535}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 65535}, + // Implicit local timestamp. + encoded: "666f6f0000000000000000000000ffff0d", + }, + "walltime and logical, smaller local walltime": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652544}, + encoded: "666f6f0016cf10bc050557410000ffff16cf10bc0505574015", + }, + "walltime and logical, larger local walltime": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652546}, + // Implicit local timestamp. + encoded: "666f6f0016cf10bc050557410000ffff0d", + }, + "walltime and logical, local logical": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, + localTs: hlc.ClockTimestamp{Logical: 65535}, + encoded: "666f6f0016cf10bc050557410000ffff00000000000000000000ffff19", + }, + "walltime and logical, smaller local walltime and logical": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 65534}, + encoded: "666f6f0016cf10bc050557410000ffff16cf10bc050557410000fffe19", + }, + "walltime and logical, larger local walltime and logical": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 65536}, + // Implicit local timestamp. + encoded: "666f6f0016cf10bc050557410000ffff0d", + }, + "walltime, logical, and synthetic, smaller local walltime and logical": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535, Synthetic: true}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 65534}, + // Implicit local timestamp. + encoded: "666f6f0016cf10bc050557410000ffff010e", + }, + "walltime, logical, and synthetic, larger local walltime and logical": { + key: "foo", + ts: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535, Synthetic: true}, + localTs: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 65536}, + // Implicit local timestamp. + encoded: "666f6f0016cf10bc050557410000ffff010e", + }, } for name, tc := range testcases { t.Run(name, func(t *testing.T) { @@ -187,17 +370,26 @@ func TestEncodeDecodeMVCCKeyAndTimestampWithLength(t *testing.T) { expect = nil } - mvccKey := MVCCKey{Key: []byte(tc.key), Timestamp: tc.ts} + mvccKey := MVCCKey{ + Key: []byte(tc.key), + Timestamp: tc.ts, + LocalTimestamp: tc.localTs, + } encoded := EncodeMVCCKey(mvccKey) require.Equal(t, expect, encoded) require.Equal(t, len(encoded), encodedMVCCKeyLength(mvccKey)) require.Equal(t, len(encoded), - encodedMVCCKeyPrefixLength(mvccKey.Key)+encodedMVCCTimestampSuffixLength(mvccKey.Timestamp)) + encodedMVCCKeyPrefixLength(mvccKey.Key)+ + encodedMVCCTimestampSuffixLength(mvccKey.Timestamp, mvccKey.LocalTimestamp)) + expDecoded := mvccKey.Normalize() decoded, err := DecodeMVCCKey(encoded) require.NoError(t, err) - require.Equal(t, mvccKey, decoded) + require.Equal(t, expDecoded, decoded) + + encoded = EncodeMVCCKey(decoded) + require.Equal(t, expect, encoded) // Test EncodeMVCCKeyPrefix. expectPrefix, err := hex.DecodeString(tc.encoded[:2*len(tc.key)+2]) @@ -212,11 +404,11 @@ func TestEncodeDecodeMVCCKeyAndTimestampWithLength(t *testing.T) { expectTS = nil } - encodedTS := EncodeMVCCTimestampSuffix(tc.ts) + encodedTS := EncodeMVCCTimestampSuffix(tc.ts, tc.localTs) require.Equal(t, expectTS, encodedTS) - require.Equal(t, len(encodedTS), encodedMVCCTimestampSuffixLength(tc.ts)) + require.Equal(t, len(encodedTS), encodedMVCCTimestampSuffixLength(tc.ts, tc.localTs)) - decodedTS, err := decodeMVCCTimestampSuffix(encodedTS) + decodedTS, _, err := decodeMVCCTimestampSuffix(encodedTS) require.NoError(t, err) require.Equal(t, tc.ts, decodedTS) @@ -225,11 +417,11 @@ func TestEncodeDecodeMVCCKeyAndTimestampWithLength(t *testing.T) { expectTS = expectTS[:len(expectTS)-1] } - encodedTS = encodeMVCCTimestamp(tc.ts) + encodedTS = encodeMVCCTimestamp(tc.ts, tc.localTs) require.Equal(t, expectTS, encodedTS) - require.Equal(t, len(encodedTS), encodedMVCCTimestampLength(tc.ts)) + require.Equal(t, len(encodedTS), encodedMVCCTimestampLength(tc.ts, tc.localTs)) - decodedTS, err = decodeMVCCTimestamp(encodedTS) + decodedTS, _, err = decodeMVCCTimestamp(encodedTS) require.NoError(t, err) require.Equal(t, tc.ts, decodedTS) }) @@ -245,18 +437,62 @@ func TestDecodeUnnormalizedMVCCKey(t *testing.T) { equalToNormal bool }{ "zero logical": { - encoded: "666f6f0016cf10bc05055741000000000d", - expected: MVCCKey{Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 0}}, + encoded: "666f6f0016cf10bc05055741000000000d", + expected: MVCCKey{ + Key: []byte("foo"), + Timestamp: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 0}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 0}, + }, equalToNormal: true, }, "zero walltime and logical": { - encoded: "666f6f000000000000000000000000000d", - expected: MVCCKey{Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 0, Logical: 0}}, + encoded: "666f6f000000000000000000000000000d", + expected: MVCCKey{ + Key: []byte("foo"), + Timestamp: hlc.Timestamp{WallTime: 0, Logical: 0}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 0, Logical: 0}, + }, // We could normalize this form in EngineKeyEqual and EngineKeyCompare, // but doing so is not worth losing the fast-path byte comparison between // keys that only contain (at most) a walltime. equalToNormal: false, }, + "zero local logical": { + encoded: "666f6f0016cf10bc050557410000ffff16cf10bc050557410000000019", + expected: MVCCKey{ + Key: []byte("foo"), + Timestamp: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 0}, + }, + equalToNormal: true, + }, + "zero local walltime and logical": { + encoded: "666f6f0016cf10bc050557410000ffff00000000000000000000000019", + expected: MVCCKey{ + Key: []byte("foo"), + Timestamp: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 0, Logical: 0}, + }, + equalToNormal: true, + }, + "larger local walltime": { + encoded: "666f6f0016cf10bc050557410000ffff16cf10bc0505574215", + expected: MVCCKey{ + Key: []byte("foo"), + Timestamp: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652546, Logical: 0}, + }, + equalToNormal: true, + }, + "larger local walltime and logical": { + encoded: "666f6f0016cf10bc050557410000ffff16cf10bc050557420000ffff19", + expected: MVCCKey{ + Key: []byte("foo"), + Timestamp: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652546, Logical: 65535}, + }, + equalToNormal: true, + }, } for name, tc := range testcases { t.Run(name, func(t *testing.T) { @@ -276,6 +512,29 @@ func TestDecodeUnnormalizedMVCCKey(t *testing.T) { } } +// TestEncodeDecodeMVCCKeySyntheticAndLocalDoesNotRoundTrip demonstrates that an +// MVCC key that has an MVCC timestamp with a synthetic bit and a local timestamp +// does not perfectly round trip. This is intentional, as the synthetic bit +// supersedes the local timestamp. +func TestEncodeDecodeMVCCKeySyntheticAndLocalDoesNotRoundTrip(t *testing.T) { + defer leaktest.AfterTest(t)() + + mvccKey := MVCCKey{ + Key: []byte("key"), + Timestamp: hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535, Synthetic: true}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 65535}, + } + + encoded := EncodeMVCCKey(mvccKey) + decoded, err := DecodeMVCCKey(encoded) + require.NoError(t, err) + + // The key does not round trip because it drops the local timestamp. + require.NotEqual(t, mvccKey, decoded) + mvccKey.LocalTimestamp = hlc.MinClockTimestamp + require.Equal(t, mvccKey, decoded) +} + func TestDecodeMVCCKeyErrors(t *testing.T) { defer leaktest.AfterTest(t)() @@ -315,7 +574,7 @@ func TestDecodeMVCCTimestampSuffixErrors(t *testing.T) { encoded, err := hex.DecodeString(tc.encoded) require.NoError(t, err) - _, err = decodeMVCCTimestampSuffix(encoded) + _, _, err = decodeMVCCTimestampSuffix(encoded) require.Error(t, err) require.Contains(t, err.Error(), tc.expectErr) }) @@ -331,20 +590,26 @@ func BenchmarkEncodeMVCCKey(b *testing.B) { "long": bytes.Repeat([]byte{1}, 4096), } timestamps := map[string]hlc.Timestamp{ - "empty": {}, - "walltime": {WallTime: 1643550788737652545}, - "walltime+logical": {WallTime: 1643550788737652545, Logical: 4096}, - "all": {WallTime: 1643550788737652545, Logical: 4096, Synthetic: true}, + "empty": {}, + "walltime": {WallTime: 1643550788737652545}, + "walltime+logical": {WallTime: 1643550788737652545, Logical: 4096}, + "walltime+logical+synthetic": {WallTime: 1643550788737652545, Logical: 4096, Synthetic: true}, } buf := make([]byte, 0, 65536) for keyDesc, key := range keys { for tsDesc, ts := range timestamps { - mvccKey := MVCCKey{Key: key, Timestamp: ts} - b.Run(fmt.Sprintf("key=%s/ts=%s", keyDesc, tsDesc), func(b *testing.B) { - for i := 0; i < b.N; i++ { - EncodeMVCCKeyToBuf(buf, mvccKey) + for localTsDesc, localTs := range timestamps { + if localTs.Synthetic { + continue // local timestamp cannot be synthetic } - }) + mvccKey := MVCCKey{Key: key, Timestamp: ts, LocalTimestamp: hlc.ClockTimestamp(localTs)} + name := fmt.Sprintf("key=%s/ts=%s/localTs=%s", keyDesc, tsDesc, localTsDesc) + b.Run(name, func(b *testing.B) { + for i := 0; i < b.N; i++ { + EncodeMVCCKeyToBuf(buf, mvccKey) + } + }) + } } } benchmarkEncodeMVCCKeyResult = buf // avoid compiler optimizing away function call @@ -359,24 +624,32 @@ func BenchmarkDecodeMVCCKey(b *testing.B) { "long": bytes.Repeat([]byte{1}, 4096), } timestamps := map[string]hlc.Timestamp{ - "empty": {}, - "walltime": {WallTime: 1643550788737652545}, - "walltime+logical": {WallTime: 1643550788737652545, Logical: 4096}, - "all": {WallTime: 1643550788737652545, Logical: 4096, Synthetic: true}, + "empty": {}, + "walltime": {WallTime: 1643550788737652545}, + "walltime+logical": {WallTime: 1643550788737652545, Logical: 4096}, + "walltime+logical+synthetic": {WallTime: 1643550788737652545, Logical: 4096, Synthetic: true}, } var mvccKey MVCCKey var err error for keyDesc, key := range keys { for tsDesc, ts := range timestamps { - encoded := EncodeMVCCKey(MVCCKey{Key: key, Timestamp: ts}) - b.Run(fmt.Sprintf("key=%s/ts=%s", keyDesc, tsDesc), func(b *testing.B) { - for i := 0; i < b.N; i++ { - mvccKey, err = DecodeMVCCKey(encoded) - if err != nil { // for performance - require.NoError(b, err) - } + for localTsDesc, localTs := range timestamps { + if localTs.Synthetic { + continue // local timestamp cannot be synthetic } - }) + encoded := EncodeMVCCKey(MVCCKey{ + Key: key, Timestamp: ts, LocalTimestamp: hlc.ClockTimestamp(localTs), + }) + name := fmt.Sprintf("key=%s/ts=%s/localTs=%s", keyDesc, tsDesc, localTsDesc) + b.Run(name, func(b *testing.B) { + for i := 0; i < b.N; i++ { + mvccKey, err = DecodeMVCCKey(encoded) + if err != nil { // for performance + require.NoError(b, err) + } + } + }) + } } } benchmarkDecodeMVCCKeyResult = mvccKey // avoid compiler optimizing away function call diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 855d5e9fe01d..77d304289fa4 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -300,8 +300,10 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { ); err != nil { t.Fatal(err) } - // Account for removal of TxnDidNotUpdateMeta + // Account for removal of TxnDidNotUpdateMeta. mValSize -= 2 + // Account for addition of LocalTimestamp. + mValSize += 8 expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 4e9, diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 72a91e7b24d9..268e56fbf0e3 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -960,23 +960,23 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := engine.PutMVCC(MVCCKey{testKey1, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err := engine.PutMVCC(MVCCKey{Key: testKey1, Timestamp: hlc.Timestamp{WallTime: 5}}, []byte("foobar")) 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}}, []byte("foobar")) 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}}, []byte("foobar")) 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}}, []byte("foobar")) 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}}, []byte("foobar")) if err != nil { t.Fatal(err) } @@ -997,7 +997,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. diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 6bad3dfeeb7b..361c484ebf9f 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -200,10 +200,18 @@ func normalizeEngineKeyVersionForCompare(a []byte) []byte { const withWall = mvccEncodedTimeSentinelLen + mvccEncodedTimeWallLen const withLogical = withWall + mvccEncodedTimeLogicalLen const withSynthetic = withLogical + mvccEncodedTimeSyntheticLen - if len(a) == withSynthetic { - // Strip the synthetic bit component from the timestamp version. The - // presence of the synthetic bit does not affect key ordering or equality. - a = a[:withLogical] + const withLocalWall = withLogical + mvccEncodedTimeWallLen + const withLocalLogical = withLocalWall + mvccEncodedTimeLogicalLen + if len(a) > withLogical { + if len(a) == withSynthetic || len(a) == withLocalWall || len(a) == withLocalLogical { + // If the timestamp is synthetic, strip the synthetic bit component from + // the timestamp version. The presence of the synthetic bit does not + // affect key ordering or equality. Similarly, if the version contains a + // local timestamp, with or without a logical component, strip it. These + // conditions are mutually exclusive, as a key with a synthetic timestamp + // will never include a local timestamp. + a = a[:withLogical] + } } if len(a) == withLogical { // If the timestamp version contains a logical timestamp component that is @@ -387,7 +395,7 @@ func (t *pebbleTimeBoundPropCollector) Finish(userProps map[string]string) error return nil //nolint:returnerrcheck } if meta.Txn != nil { - ts := encodeMVCCTimestamp(meta.Timestamp.ToTimestamp()) + ts := encodeMVCCTimestamp(meta.Timestamp.ToTimestamp(), hlc.ClockTimestamp{}) t.updateBounds(ts) } } @@ -1578,8 +1586,8 @@ func (p *Pebble) Compact() error { // CompactRange implements the Engine interface. func (p *Pebble) CompactRange(start, end roachpb.Key) error { - bufStart := EncodeMVCCKey(MVCCKey{start, hlc.Timestamp{}}) - bufEnd := EncodeMVCCKey(MVCCKey{end, hlc.Timestamp{}}) + bufStart := EncodeMVCCKey(MakeMVCCMetadataKey(start)) + bufEnd := EncodeMVCCKey(MakeMVCCMetadataKey(end)) return p.db.Compact(bufStart, bufEnd, true /* parallel */) } diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index 8fc315668657..2cff5e22e3ee 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "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/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -148,8 +149,8 @@ func (p *pebbleIterator) init( doClone := iterToClone != nil if !opts.MaxTimestampHint.IsEmpty() { doClone = false - encodedMinTS := string(encodeMVCCTimestamp(opts.MinTimestampHint)) - encodedMaxTS := string(encodeMVCCTimestamp(opts.MaxTimestampHint)) + encodedMinTS := string(encodeMVCCTimestamp(opts.MinTimestampHint, hlc.ClockTimestamp{})) + encodedMaxTS := string(encodeMVCCTimestamp(opts.MaxTimestampHint, hlc.ClockTimestamp{})) p.options.TableFilter = func(userProps map[string]string) bool { tableMinTS := userProps["crdb.ts.min"] if len(tableMinTS) == 0 { diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index b5a4ecb12e09..15c10ffa8ec7 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -595,7 +595,7 @@ 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) { + if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp, p.curUnsafeKey.LocalTimestamp) { return p.uncertaintyError(p.curUnsafeKey.Timestamp) } @@ -655,7 +655,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // we want to read the intent regardless of our read timestamp and fall // into case 11 below. if p.checkUncertainty { - if p.uncertainty.IsUncertain(metaTS) { + if p.uncertainty.IsUncertain(metaTS, p.meta.GetLocalTimestamp()) { return p.uncertaintyError(metaTS) } // The intent is not within the uncertainty window, but there could @@ -1012,7 +1012,7 @@ 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) { + if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp, p.curUnsafeKey.LocalTimestamp) { return p.uncertaintyError(p.curUnsafeKey.Timestamp) } } @@ -1032,7 +1032,7 @@ func (p *pebbleMVCCScanner) seekVersion( // 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) { + if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp, p.curUnsafeKey.LocalTimestamp) { return p.uncertaintyError(p.curUnsafeKey.Timestamp) } if !p.iterNext() { diff --git a/pkg/storage/pebble_mvcc_scanner_test.go b/pkg/storage/pebble_mvcc_scanner_test.go index 597eb9a21c7a..4d66071ddeab 100644 --- a/pkg/storage/pebble_mvcc_scanner_test.go +++ b/pkg/storage/pebble_mvcc_scanner_test.go @@ -113,7 +113,11 @@ 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].k = MVCCKey{ + Key: keys[i], + Timestamp: hlc.Timestamp{WallTime: 2}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 2}, + } expectedKVs[i].v = []byte("2") } require.Equal(t, expectedKVs, kvs) diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index c1e95b8a93d7..ca4b57a64ff9 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -165,7 +165,7 @@ 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}} + key := MVCCKey{Key: []byte{byte(i)}, Timestamp: hlc.Timestamp{WallTime: 100}} if err := batch.PutMVCC(key, []byte("foo")); err != nil { t.Fatal(err) } @@ -496,7 +496,7 @@ func TestPebbleIterConsistency(t *testing.T) { defer eng.Close() ts1 := hlc.Timestamp{WallTime: 1} ts2 := hlc.Timestamp{WallTime: 2} - k1 := MVCCKey{[]byte("a"), ts1} + k1 := MVCCKey{Key: []byte("a"), Timestamp: ts1} require.NoError(t, eng.PutMVCC(k1, []byte("a1"))) var ( @@ -527,7 +527,7 @@ 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"))) + require.NoError(t, eng.PutMVCC(MVCCKey{Key: []byte("a"), Timestamp: ts2}, []byte("a2"))) checkMVCCIter := func(iter MVCCIterator) { defer iter.Close() @@ -594,8 +594,8 @@ 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}, []byte("a2"))) + require.NoError(t, batch2.PutMVCC(MVCCKey{Key: []byte("a"), Timestamp: ts2}, []byte("a2"))) checkIterSeesBothValues(batch.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: []byte("b")})) checkIterSeesBothValues(batch2.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: []byte("b")})) } @@ -634,6 +634,14 @@ func makeRandEncodedKeys() [][]byte { // 0.1% of keys have a synthetic component. k.Timestamp.Synthetic = true } + if rng.Int31n(100) < 3 { + // 3% of keys have a local timestamp. + k.LocalTimestamp.WallTime = rng.Int63n(5) + if rng.Int31n(5) == 0 { + // Of which, 20% have a logical component to the local timestamp. + k.LocalTimestamp.Logical = rng.Int31n(4) + 1 + } + } keys[i] = EncodeMVCCKey(k) } return keys @@ -881,7 +889,7 @@ 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}}, []byte("a"))) require.NoError(t, eng.db.Flush()) } @@ -1127,33 +1135,33 @@ func TestPebbleMVCCTimeIntervalCollector(t *testing.T) { // The added key was not an MVCCKey. finishAndCheck(0, 0) require.NoError(t, collector.Add(pebble.InternalKey{ - UserKey: EncodeMVCCKey(MVCCKey{aKey, hlc.Timestamp{WallTime: 2, Logical: 1}})}, + UserKey: EncodeMVCCKey(MVCCKey{Key: aKey, Timestamp: hlc.Timestamp{WallTime: 2, Logical: 1}})}, []byte("foo"))) // Added 1 MVCCKey which sets both the upper and lower bound. finishAndCheck(2, 3) require.NoError(t, collector.Add(pebble.InternalKey{ - UserKey: EncodeMVCCKey(MVCCKey{aKey, hlc.Timestamp{WallTime: 22, Logical: 1}})}, + UserKey: EncodeMVCCKey(MVCCKey{Key: aKey, Timestamp: hlc.Timestamp{WallTime: 22, Logical: 1}})}, []byte("foo"))) require.NoError(t, collector.Add(pebble.InternalKey{ - UserKey: EncodeMVCCKey(MVCCKey{aKey, hlc.Timestamp{WallTime: 25, Logical: 1}})}, + UserKey: EncodeMVCCKey(MVCCKey{Key: aKey, Timestamp: hlc.Timestamp{WallTime: 25, Logical: 1}})}, []byte("foo"))) // Added 2 MVCCKeys. finishAndCheck(22, 26) // Using the same suffix for all keys in a block results in an interval of // width one (inclusive lower bound to exclusive upper bound). - suffix := EncodeMVCCTimestampSuffix(hlc.Timestamp{WallTime: 42, Logical: 1}) + suffix := EncodeMVCCTimestampSuffix(hlc.Timestamp{WallTime: 42, Logical: 1}, hlc.ClockTimestamp{}) require.NoError(t, collector.UpdateKeySuffixes( nil /* old prop */, nil /* old suffix */, suffix, )) finishAndCheck(42, 43) // An invalid key results in an error. // Case 1: malformed sentinel. - key := EncodeMVCCKey(MVCCKey{aKey, hlc.Timestamp{WallTime: 2, Logical: 1}}) + key := EncodeMVCCKey(MVCCKey{Key: aKey, Timestamp: hlc.Timestamp{WallTime: 2, Logical: 1}}) sentinelPos := len(key) - 1 - int(key[len(key)-1]) key[sentinelPos] = '\xff' require.Error(t, collector.UpdateKeySuffixes(nil, nil, key)) // Case 2: malformed bare suffix (too short). - suffix = EncodeMVCCTimestampSuffix(hlc.Timestamp{WallTime: 42, Logical: 1})[1:] + suffix = EncodeMVCCTimestampSuffix(hlc.Timestamp{WallTime: 42, Logical: 1}, hlc.ClockTimestamp{})[1:] require.Error(t, collector.UpdateKeySuffixes(nil, nil, suffix)) } @@ -1174,7 +1182,7 @@ 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}}, + MVCCKey{Key: aKey, Timestamp: hlc.Timestamp{WallTime: int64(i), Logical: 1}}, []byte(fmt.Sprintf("val%d", i)))) } require.NoError(t, eng.Flush()) @@ -1205,7 +1213,7 @@ 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} // Write. require.NoError(t, eng.PutMVCC(k, []byte("a1"))) cbCount := int32(0) diff --git a/pkg/storage/sst.go b/pkg/storage/sst.go index b75253c4a066..5e5f36bc2556 100644 --- a/pkg/storage/sst.go +++ b/pkg/storage/sst.go @@ -247,8 +247,8 @@ func UpdateSSTTimestamps( opts, sstOut, MakeIngestionWriterOptions(ctx, st), - EncodeMVCCTimestampSuffix(from), - EncodeMVCCTimestampSuffix(to), + EncodeMVCCTimestampSuffix(from, hlc.ClockTimestamp{}), + EncodeMVCCTimestampSuffix(to, hlc.ClockTimestamp{}), concurrency, ); err != nil { return nil, err 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..ad2257bccda5 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@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@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..3a198605a16c 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 localTs=11.000000000,0 del=false klen=12 vlen=6 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 localTs=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +data: "k"/14.000000000,0@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 localTs=11.000000000,0 del=false klen=12 vlen=6 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@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@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@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@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@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@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 localTs=40.000000000,0 del=false klen=12 vlen=6 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@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 localTs=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false +data: "n"/45.000000000,0@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 localTs=40.000000000,0 del=false klen=12 vlen=6 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@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@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@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@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@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@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@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@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@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@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@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@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@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@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..d92e6a3fcc90 --- /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@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@10.000000000,0 -> /BYTES/v +data: "k2"/30.000000000,0 -> /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/30.000000000,0@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@30.000000000,0 -> / +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0@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@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@30.000000000,0 -> / +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0@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@40.000000000,0 -> /INT/1 +data: "k3"/40.000000000,0 -> / +data: "k3"/30.000000000,0@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@50.000000000,0 -> / +data: "k1"/50.000000000,0 -> /INT/1 +data: "k1"/40.000000000,0@30.000000000,0 -> / +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0@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@40.000000000,0 -> /INT/1 +data: "k3"/40.000000000,0 -> / +data: "k3"/30.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k1"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k10"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k11"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k12"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k2"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k3"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k4"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k8"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=true +data: "k9"/20.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k1"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k10"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k11"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k12"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k2"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k3"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k4"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k5"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k6"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k7"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k8"/30.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k9"/30.000000000,0@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@20.000000000,0 -> /BYTES/v2 +data: "k11"/40.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k5"/40.000000000,0@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 localTs=20.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k6"/40.000000000,0@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 localTs=30.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k7"/40.000000000,0@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@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..d5f0c9c10a8a 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@5 # -# k3: value @ ts 20? +# k3: value @ ts 20@10 # k3: value @ ts 10 # -# k4: value @ ts 20? -# k4: value @ ts 10? +# k4: value @ ts 20@10 +# k4: value @ ts 10@5 # # k5: intent @ ts 20 # k5: value @ ts 10 # # k6: intent @ ts 20 -# k6: value @ ts 10? +# k6: value @ ts 10@5 # -# k7: intent @ ts 20? +# k7: intent @ ts 20@10 # k7: value @ ts 10 # -# k8: intent @ ts 20? -# k8: value @ ts 10? +# k8: intent @ ts 20@10 +# k8: value @ ts 10@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@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@5.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0@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@5.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0@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@10.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0@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@5.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0@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@10.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0@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@5.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0@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@10.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0@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@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@5.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0@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@10.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0@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@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 localTs=10.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0@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@5.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0@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@10.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0@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@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 localTs=10.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0@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 localTs=10.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k8"/20.000000000,0@10.000000000,0 -> /BYTES/v16 +data: "k8"/10.000000000,0@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/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.