diff --git a/pkg/ccl/backupccl/backup_metadata.go b/pkg/ccl/backupccl/backup_metadata.go index fee3e840d639..9d3712465512 100644 --- a/pkg/ccl/backupccl/backup_metadata.go +++ b/pkg/ccl/backupccl/backup_metadata.go @@ -187,7 +187,7 @@ func writeDescsToMetadata(ctx context.Context, sst storage.SSTWriter, m *BackupM b = bytes } } - if err := sst.PutMVCC(storage.MVCCKey{Key: k, Timestamp: i.Time}, b); err != nil { + if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: i.Time}, b); err != nil { return err } @@ -214,7 +214,7 @@ func writeDescsToMetadata(ctx context.Context, sst storage.SSTWriter, m *BackupM return err } } else { - if err := sst.PutMVCC(storage.MVCCKey{Key: k, Timestamp: m.StartTime}, b); err != nil { + if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: m.StartTime}, b); err != nil { return err } } @@ -340,7 +340,7 @@ func writeNamesToMetadata(ctx context.Context, sst storage.SSTWriter, m *BackupM } k := encodeNameSSTKey(rev.parent, rev.parentSchema, rev.name) v := encoding.EncodeUvarintAscending(nil, uint64(rev.id)) - if err := sst.PutMVCC(storage.MVCCKey{Key: k, Timestamp: rev.ts}, v); err != nil { + if err := sst.PutRawMVCC(storage.MVCCKey{Key: k, Timestamp: rev.ts}, v); err != nil { return err } } @@ -384,7 +384,7 @@ func writeSpansToMetadata(ctx context.Context, sst storage.SSTWriter, m *BackupM } } else { k := storage.MVCCKey{Key: encodeSpanSSTKey(sp), Timestamp: ts} - if err := sst.PutMVCC(k, nil); err != nil { + if err := sst.PutRawMVCC(k, nil); err != nil { return err } } diff --git a/pkg/ccl/backupccl/backup_processor.go b/pkg/ccl/backupccl/backup_processor.go index bc328d75aa7b..2120b7d9af9b 100644 --- a/pkg/ccl/backupccl/backup_processor.go +++ b/pkg/ccl/backupccl/backup_processor.go @@ -846,7 +846,7 @@ func (s *sstSink) write(ctx context.Context, resp returnedSST) error { return err } } else { - if err := s.sst.PutMVCC(sst.UnsafeKey(), sst.UnsafeValue()); err != nil { + if err := s.sst.PutRawMVCC(sst.UnsafeKey(), sst.UnsafeValue()); err != nil { return err } } diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index d95ef1b9bb38..6eaac8b9892c 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -476,9 +476,10 @@ func (rd *restoreDataProcessor) processRestoreSpanEntry( continue } - keyScratch = append(keyScratch[:0], iter.UnsafeKey().Key...) + key := iter.UnsafeKey() + keyScratch = append(keyScratch[:0], key.Key...) + key.Key = keyScratch valueScratch = append(valueScratch[:0], iter.UnsafeValue()...) - key := storage.MVCCKey{Key: keyScratch, Timestamp: iter.UnsafeKey().Timestamp} value := roachpb.Value{RawBytes: valueScratch} iter.NextKey() diff --git a/pkg/ccl/backupccl/restore_data_processor_test.go b/pkg/ccl/backupccl/restore_data_processor_test.go index ea2caed2b9a0..1c929cb5bc60 100644 --- a/pkg/ccl/backupccl/restore_data_processor_test.go +++ b/pkg/ccl/backupccl/restore_data_processor_test.go @@ -81,20 +81,23 @@ func slurpSSTablesLatestKey( if !sst.UnsafeKey().Less(end) { break } - var ok bool - var newKv storage.MVCCKeyValue key := sst.UnsafeKey() - newKv.Value = append(newKv.Value, sst.UnsafeValue()...) - newKv.Key.Key = append(newKv.Key.Key, key.Key...) - newKv.Key.Timestamp = key.Timestamp - newKv.Key.Key, ok = kr.rewriteKey(newKv.Key.Key) + value, err := storage.DecodeMVCCValue(sst.UnsafeValue()) + if err != nil { + t.Fatal(err) + } + newKey := key + newKey.Key = append([]byte(nil), newKey.Key...) + var ok bool + newKey.Key, ok = kr.rewriteKey(newKey.Key) if !ok { - t.Fatalf("could not rewrite key: %s", newKv.Key.Key) + t.Fatalf("could not rewrite key: %s", newKey.Key) } - v := roachpb.Value{RawBytes: newKv.Value} - v.ClearChecksum() - v.InitChecksum(newKv.Key.Key) - if err := batch.PutMVCC(newKv.Key, v.RawBytes); err != nil { + newValue := value + newValue.Value.RawBytes = append([]byte(nil), newValue.Value.RawBytes...) + newValue.Value.ClearChecksum() + newValue.Value.InitChecksum(newKey.Key) + if err := batch.PutMVCC(newKey, newValue); err != nil { t.Fatal(err) } sst.Next() diff --git a/pkg/cli/debug_check_store_test.go b/pkg/cli/debug_check_store_test.go index 0f1df93c5498..4153ef5049e8 100644 --- a/pkg/cli/debug_check_store_test.go +++ b/pkg/cli/debug_check_store_test.go @@ -75,7 +75,7 @@ func TestDebugCheckStore(t *testing.T) { // Should not error out randomly. for _, dir := range storePaths { out, err := check(dir) - require.NoError(t, err, dir) + require.NoError(t, err, "dir=%s\nout=%s\n", dir, out) require.Contains(t, out, "total stats", dir) } diff --git a/pkg/cmd/roachtest/tests/acceptance.go b/pkg/cmd/roachtest/tests/acceptance.go index d489c182be63..c0c45fb1efb1 100644 --- a/pkg/cmd/roachtest/tests/acceptance.go +++ b/pkg/cmd/roachtest/tests/acceptance.go @@ -52,6 +52,7 @@ func registerAcceptance(r registry.Registry) { }, { name: "version-upgrade", + skip: "WIP: unskip when version checks are added to local_timestamp writes", fn: func(ctx context.Context, t test.Test, c cluster.Cluster) { runVersionUpgrade(ctx, t, c) }, diff --git a/pkg/kv/kvnemesis/engine.go b/pkg/kv/kvnemesis/engine.go index f6cc5391c789..257acba4b4e3 100644 --- a/pkg/kv/kvnemesis/engine.go +++ b/pkg/kv/kvnemesis/engine.go @@ -67,12 +67,18 @@ func (e *Engine) Get(key roachpb.Key, ts hlc.Timestamp) roachpb.Value { if !mvccKey.Key.Equal(key) { return roachpb.Value{} } - if len(iter.Value()) == 0 { - return roachpb.Value{} - } var valCopy []byte e.b, valCopy = e.b.Copy(iter.Value(), 0 /* extraCap */) - return roachpb.Value{RawBytes: valCopy, Timestamp: mvccKey.Timestamp} + mvccVal, err := storage.DecodeMVCCValue(valCopy) + if err != nil { + panic(err) + } + if mvccVal.IsTombstone() { + return roachpb.Value{} + } + val := mvccVal.Value + val.Timestamp = mvccKey.Timestamp + return val } // Put inserts a key/value/timestamp tuple. If an exact key/timestamp pair is @@ -124,8 +130,13 @@ func (e *Engine) DebugPrint(indent string) string { if err != nil { fmt.Fprintf(&buf, "(err:%s)", err) } else { - fmt.Fprintf(&buf, "%s%s %s -> %s", - indent, key.Key, key.Timestamp, roachpb.Value{RawBytes: value}.PrettyPrint()) + v, err := storage.DecodeMVCCValue(value) + if err != nil { + fmt.Fprintf(&buf, "(err:%s)", err) + } else { + fmt.Fprintf(&buf, "%s%s %s -> %s", + indent, key.Key, key.Timestamp, v.Value.PrettyPrint()) + } } }) return buf.String() diff --git a/pkg/kv/kvnemesis/validator.go b/pkg/kv/kvnemesis/validator.go index 6ce9fab5e01b..c8d1c95b1523 100644 --- a/pkg/kv/kvnemesis/validator.go +++ b/pkg/kv/kvnemesis/validator.go @@ -273,8 +273,12 @@ func makeValidator(kvs *Engine) (*validator, error) { err = errors.CombineErrors(err, iterErr) return } - v := roachpb.Value{RawBytes: value} - if v.GetTag() != roachpb.ValueType_UNKNOWN { + v, decodeErr := storage.DecodeMVCCValue(value) + if err != nil { + err = errors.CombineErrors(err, decodeErr) + return + } + if v.Value.GetTag() != roachpb.ValueType_UNKNOWN { valueStr := mustGetStringValue(value) if existing, ok := kvByValue[valueStr]; ok { // TODO(dan): This may be too strict. Some operations (db.Run on a @@ -287,7 +291,7 @@ func makeValidator(kvs *Engine) (*validator, error) { // globally over a run, so there's a 1:1 relationship between a value that // was written and the operation that wrote it. kvByValue[valueStr] = storage.MVCCKeyValue{Key: key, Value: value} - } else if len(value) == 0 { + } else if !v.Value.IsPresent() { rawKey := string(key.Key) if _, ok := tombstonesForKey[rawKey]; !ok { tombstonesForKey[rawKey] = make(map[hlc.Timestamp]bool) @@ -914,14 +918,18 @@ func resultIsErrorStr(r Result, msgRE string) bool { } func mustGetStringValue(value []byte) string { - if len(value) == 0 { + v, err := storage.DecodeMVCCValue(value) + if err != nil { + panic(errors.Wrapf(err, "decoding %x", value)) + } + if v.IsTombstone() { return `` } - v, err := roachpb.Value{RawBytes: value}.GetBytes() + b, err := v.Value.GetBytes() if err != nil { panic(errors.Wrapf(err, "decoding %x", value)) } - return string(v) + return string(b) } func validReadTimes( diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index 75cd43a52c0e..4c0eba446b09 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -335,7 +335,7 @@ func EvalAddSSTable( return result.Result{}, err } } else { - if err := readWriter.PutMVCC(k, sstIter.UnsafeValue()); err != nil { + if err := readWriter.PutRawMVCC(k, sstIter.UnsafeValue()); err != nil { return result.Result{}, err } } @@ -401,11 +401,15 @@ func assertSSTContents(sst []byte, sstTimestamp hlc.Timestamp, stats *enginepb.M break } - key, value := iter.UnsafeKey(), iter.UnsafeValue() + key, valueRaw := iter.UnsafeKey(), iter.UnsafeValue() + value, err := storage.DecodeMVCCValue(valueRaw) + if err != nil { + return err + } if key.Timestamp.IsEmpty() { return errors.AssertionFailedf("SST contains inline value or intent for key %s", key) } - if len(value) == 0 { + if value.IsTombstone() { return errors.AssertionFailedf("SST contains tombstone for key %s", key) } if sstTimestamp.IsSet() && key.Timestamp != sstTimestamp { diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 7635e89a539e..b29e721d04bf 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -1117,7 +1117,7 @@ func TestAddSSTableMVCCStats(t *testing.T) { {"e", 1, "e"}, {"z", 2, "zzzzzz"}, } { - require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) } sst, start, end := sstutil.MakeSST(t, st, []sstutil.KV{ @@ -1228,7 +1228,7 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { {"y", 5, "yyy"}, {"z", 2, "zz"}, } { - require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) } // This test ensures accuracy of MVCCStats in the situation that successive @@ -1270,7 +1270,7 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { // ingesting the perfectly shadowing KVs (same ts and same value) in the // second SST. for _, kv := range kvs { - require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) } // Evaluate the second SST. Both the KVs are perfectly shadowing and should diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 85a73f746266..31216da2153d 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -492,6 +492,253 @@ func TestTxnReadWithinUncertaintyInterval(t *testing.T) { }) } +// TestTxnReadWithinUncertaintyIntervalAfterIntentResolution tests cases where a +// reader transaction observes a committed value that was committed before the +// reader began, but that was resolved after the reader began. The test ensures +// that even if the reader has collected an observed timestamp from the node +// that holds the intent, and even if this observed timestamp is less than the +// timestamp that the intent is eventually committed at, the reader still +// considers the value to be in its uncertainty interval. Not doing so could +// allow for stale read, which would be a violation of linearizability. +// +// This is a regression test for #36431. Before this issue was addressed, +// it was possible for the following series of events to lead to a stale +// read: +// - txn W is coordinated by node B. It lays down an intent on node A (key k) at +// ts 95. +// - txn W gets pushed to ts 105 (taken from B's clock). It refreshes +// successfully and commits at 105. Node A's clock is at, say, 100; this is +// within clock offset bounds. +// - after all this, txn R starts on node A. It gets assigned ts 100. The txn +// has no uncertainty for node A. +// - txn W's async intent resolution comes around and resolves the intent on +// node A, moving the value fwd from ts 95 to 105. +// - txn R reads key k and doesn't see anything. There's a value at 105, but the +// txn have no uncertainty due to an observed timestamp. This is a stale read. +// +// The test's rangedResolution parameter dictates whether the intent is +// asynchronously resolved using point or ranged intent resolution. +// +// The test's movedWhilePending parameter dictates whether the intent is moved +// to a higher timestamp first by a PENDING intent resolution and then COMMITTED +// at that same timestamp, or whether it is moved to a higher timestamp at the +// same time as it is COMMITTED. +// +// The test's alreadyResolved parameter dictates whether the intent is +// already resolved by the time the reader observes it, or whether the +// reader must resolve the intent itself. +// +func TestTxnReadWithinUncertaintyIntervalAfterIntentResolution(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testutils.RunTrueAndFalse(t, "rangedResolution", func(t *testing.T, rangedResolution bool) { + testutils.RunTrueAndFalse(t, "movedWhilePending", func(t *testing.T, movedWhilePending bool) { + testutils.RunTrueAndFalse(t, "alreadyResolved", func(t *testing.T, alreadyResolved bool) { + testTxnReadWithinUncertaintyIntervalAfterIntentResolution( + t, rangedResolution, movedWhilePending, alreadyResolved, + ) + }) + }) + }) +} + +func testTxnReadWithinUncertaintyIntervalAfterIntentResolution( + t *testing.T, rangedResolution, movedWhilePending, alreadyResolved bool, +) { + const numNodes = 2 + var manuals []*hlc.HybridManualClock + var clocks []*hlc.Clock + for i := 0; i < numNodes; i++ { + manuals = append(manuals, hlc.NewHybridManualClock()) + } + serverArgs := make(map[int]base.TestServerArgs) + for i := 0; i < numNodes; i++ { + serverArgs[i] = base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manuals[i].UnixNano, + }, + Store: &kvserver.StoreTestingKnobs{ + IntentResolverKnobs: kvserverbase.IntentResolverTestingKnobs{ + // Disable async intent resolution, so that the test can carefully + // control when intent resolution occurs. + DisableAsyncIntentResolution: true, + }, + }, + }, + } + } + ctx := context.Background() + tc := testcluster.StartTestCluster(t, numNodes, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgsPerNode: serverArgs, + }) + defer tc.Stopper().Stop(ctx) + + // Split off two scratch ranges. + keyA, keyB := roachpb.Key("a"), roachpb.Key("b") + tc.SplitRangeOrFatal(t, keyA) + _, keyBDesc := tc.SplitRangeOrFatal(t, keyB) + // Place key A's sole replica on node 1 and key B's sole replica on node 2. + tc.AddVotersOrFatal(t, keyB, tc.Target(1)) + tc.TransferRangeLeaseOrFatal(t, keyBDesc, tc.Target(1)) + tc.RemoveVotersOrFatal(t, keyB, tc.Target(0)) + + // Pause the servers' clocks going forward. + var maxNanos int64 + for i, m := range manuals { + m.Pause() + if cur := m.UnixNano(); cur > maxNanos { + maxNanos = cur + } + clocks = append(clocks, tc.Servers[i].Clock()) + } + // After doing so, perfectly synchronize them. + for _, m := range manuals { + m.Increment(maxNanos - m.UnixNano()) + } + + // Create a new writer transaction. + maxOffset := clocks[0].MaxOffset().Nanoseconds() + require.NotZero(t, maxOffset) + writerTxn := roachpb.MakeTransaction("test_writer", keyA, 1, clocks[0].Now(), maxOffset, int32(tc.Servers[0].NodeID())) + + // Write to key A and key B in the writer transaction. + for _, key := range []roachpb.Key{keyA, keyB} { + put := putArgs(key, []byte("val")) + resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), roachpb.Header{Txn: &writerTxn}, put) + require.Nil(t, pErr) + writerTxn.Update(resp.Header().Txn) + } + + // Move the clock on just the first server and bump the transaction commit + // timestamp to this value. The clock on the second server will trail behind. + manuals[0].Increment(100) + require.True(t, writerTxn.WriteTimestamp.Forward(clocks[0].Now())) + + // Refresh the writer transaction's timestamp. + writerTxn.ReadTimestamp.Forward(writerTxn.WriteTimestamp) + + // Commit the writer transaction. Key A will be synchronously resolved because + // it is on the same range as the transaction record. However, key B will be + // handed to the IntentResolver for asynchronous resolution. Because we + // disabled async resolution, it will not be resolved yet. + et, etH := endTxnArgs(&writerTxn, true /* commit */) + et.LockSpans = []roachpb.Span{ + {Key: keyA}, {Key: keyB}, + } + if rangedResolution { + for i := range et.LockSpans { + et.LockSpans[i].EndKey = et.LockSpans[i].Key.Next() + } + } + etResp, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), etH, et) + require.Nil(t, pErr) + writerTxn.Update(etResp.Header().Txn) + + // Create a new reader transaction. The reader uses the second server as a + // gateway, so its initial read timestamp actually trails the commit timestamp + // of the writer transaction due to clock skew between the two servers. This + // is the classic case where the reader's uncertainty interval is needed to + // avoid stale reads. Remember that the reader transaction began after the + // writer transaction committed and received an ack, so it must observe the + // writer's writes if it is to respect real-time ordering. + // + // NB: we use writerTxn.MinTimestamp instead of clocks[1].Now() so that a + // stray clock update doesn't influence the reader's read timestamp. + readerTxn := roachpb.MakeTransaction("test_reader", keyA, 1, writerTxn.MinTimestamp, maxOffset, int32(tc.Servers[1].NodeID())) + require.True(t, readerTxn.ReadTimestamp.Less(writerTxn.WriteTimestamp)) + require.False(t, readerTxn.GlobalUncertaintyLimit.Less(writerTxn.WriteTimestamp)) + + // Collect an observed timestamp from each of the nodes. We read the key + // following (Key.Next) each of the written keys to avoid conflicting with + // read values. We read keyB first to avoid advancing the clock on node 2 + // before we collect an observed timestamp from it. + // + // NOTE: this wasn't even a necessary step to hit #36431, because new + // transactions are always an observed timestamp from their own gateway node. + for i, key := range []roachpb.Key{keyB, keyA} { + get := getArgs(key.Next()) + resp, pErr := kv.SendWrappedWith(ctx, tc.Servers[1].DistSender(), roachpb.Header{Txn: &readerTxn}, get) + require.Nil(t, pErr) + require.Nil(t, resp.(*roachpb.GetResponse).Value) + readerTxn.Update(resp.Header().Txn) + require.Len(t, readerTxn.ObservedTimestamps, i+1) + } + + // Resolve the intent on key B zero, one, or two times. + { + resolveIntentArgs := func(status roachpb.TransactionStatus) roachpb.Request { + if rangedResolution { + return &roachpb.ResolveIntentRangeRequest{ + RequestHeader: roachpb.RequestHeader{Key: keyB, EndKey: keyB.Next()}, + IntentTxn: writerTxn.TxnMeta, + Status: status, + } + } else { + return &roachpb.ResolveIntentRequest{ + RequestHeader: roachpb.RequestHeader{Key: keyB}, + IntentTxn: writerTxn.TxnMeta, + Status: status, + } + } + } + + if movedWhilePending { + // First change the intent's timestamp without committing it. This + // exercises the case where the intent's timestamp is moved forward by a + // PENDING intent resolution request and kept the same when the intent is + // eventually COMMITTED. This PENDING intent resolution may still be + // evaluated after the transaction commit has been acknowledged in + // real-time, so it still needs to lead to the committed value retaining + // its original local timestamp. + // + // For instance, consider the following timeline: + // + // 1. txn W writes intent on key A @ time 10 + // 2. txn W writes intent on key B @ time 10 + // 3. high priority reader @ 15 reads key B + // 4. high priority reader pushes txn W to time 15 + // 5. txn W commits @ 15 and resolves key A synchronously + // 6. txn R begins and collects observed timestamp from key B's node @ + // time 11 + // 7. high priority reader moves intent on key B to time 15 + // 8. async intent resolution commits intent on key B, still @ time 15 + // 9. txn R reads key B with read ts 11, observed ts 11, and uncertainty + // interval [11, 21]. If step 7 updated the intent's local timestamp + // to the current time when changing its version timestamp, txn R + // could use its observed timestamp to avoid an uncertainty error, + // leading to a stale read. + // + resolve := resolveIntentArgs(roachpb.PENDING) + _, pErr = kv.SendWrapped(ctx, tc.Servers[0].DistSender(), resolve) + require.Nil(t, pErr) + } + + if alreadyResolved { + // Resolve the committed value on key B to COMMITTED. + resolve := resolveIntentArgs(roachpb.COMMITTED) + _, pErr = kv.SendWrapped(ctx, tc.Servers[0].DistSender(), resolve) + require.Nil(t, pErr) + } + } + + // Read key A and B in the reader transaction. Both should produce + // ReadWithinUncertaintyIntervalErrors. + for _, key := range []roachpb.Key{keyA, keyB} { + get := getArgs(key) + _, pErr := kv.SendWrappedWith(ctx, tc.Servers[0].DistSender(), roachpb.Header{Txn: &readerTxn}, get) + require.NotNil(t, pErr) + var rwuiErr *roachpb.ReadWithinUncertaintyIntervalError + require.True(t, errors.As(pErr.GetDetail(), &rwuiErr)) + require.Equal(t, readerTxn.ReadTimestamp, rwuiErr.ReadTimestamp) + require.Equal(t, readerTxn.GlobalUncertaintyLimit, rwuiErr.GlobalUncertaintyLimit) + require.Equal(t, readerTxn.ObservedTimestamps, rwuiErr.ObservedTimestamps) + require.Equal(t, writerTxn.WriteTimestamp, rwuiErr.ExistingTimestamp) + } +} + // TestTxnReadWithinUncertaintyIntervalAfterLeaseTransfer tests a case where a // transaction observes a committed value in its uncertainty interval that was // written under a previous leaseholder. In the test, the transaction does diff --git a/pkg/kv/kvserver/client_test.go b/pkg/kv/kvserver/client_test.go index 54405929f758..a300b9dcdc4b 100644 --- a/pkg/kv/kvserver/client_test.go +++ b/pkg/kv/kvserver/client_test.go @@ -102,6 +102,15 @@ func heartbeatArgs( }, roachpb.Header{Txn: txn} } +func endTxnArgs(txn *roachpb.Transaction, commit bool) (*roachpb.EndTxnRequest, roachpb.Header) { + return &roachpb.EndTxnRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: txn.Key, // not allowed when going through TxnCoordSender, but we're not + }, + Commit: commit, + }, roachpb.Header{Txn: txn} +} + func pushTxnArgs( pusher, pushee *roachpb.Transaction, pushType roachpb.PushTxnType, ) *roachpb.PushTxnRequest { diff --git a/pkg/kv/kvserver/gc/data_distribution_test.go b/pkg/kv/kvserver/gc/data_distribution_test.go index f032b7eb120d..ac4945c42969 100644 --- a/pkg/kv/kvserver/gc/data_distribution_test.go +++ b/pkg/kv/kvserver/gc/data_distribution_test.go @@ -52,7 +52,7 @@ func (ds dataDistribution) setupTest( if kv.Key.Timestamp.IsEmpty() { require.NoError(t, eng.PutUnversioned(kv.Key.Key, kv.Value)) } else { - require.NoError(t, eng.PutMVCC(kv.Key, kv.Value)) + require.NoError(t, eng.PutRawMVCC(kv.Key, kv.Value)) } } else { // TODO(ajwerner): Decide if using MVCCPut is worth it. @@ -84,7 +84,7 @@ func (ds dataDistribution) setupTest( func newDataDistribution( tsDist func() hlc.Timestamp, keyDist func() roachpb.Key, - valueDist func() []byte, + valueDist func() roachpb.Value, versionsPerKey func() int, intentFrac float64, totalKeys int, @@ -146,7 +146,7 @@ func newDataDistribution( } return storage.MVCCKeyValue{ Key: storage.MVCCKey{Key: key, Timestamp: ts}, - Value: valueDist(), + Value: valueDist().RawBytes, }, txn, true } } @@ -218,20 +218,24 @@ func uniformTimestampDistribution(from, to int64, rng *rand.Rand) func() hlc.Tim } // returns a uniform length random value distribution. -func uniformValueDistribution(min, max int, deleteFrac float64, rng *rand.Rand) func() []byte { +func uniformValueDistribution( + min, max int, deleteFrac float64, rng *rand.Rand, +) func() roachpb.Value { if min > max { panic(fmt.Errorf("min (%d) > max (%d)", min, max)) } n := (max - min) + 1 - return func() []byte { + return func() roachpb.Value { if rng.Float64() < deleteFrac { - return nil + return roachpb.Value{} } - value := make([]byte, min+rng.Intn(n)) - if _, err := rng.Read(value); err != nil { + b := make([]byte, min+rng.Intn(n)) + if _, err := rng.Read(b); err != nil { panic(err) } - return value + var v roachpb.Value + v.SetBytes(b) + return v } } diff --git a/pkg/kv/kvserver/gc/gc_iterator_test.go b/pkg/kv/kvserver/gc/gc_iterator_test.go index 2f4533436a1e..8c148f30d82e 100644 --- a/pkg/kv/kvserver/gc/gc_iterator_test.go +++ b/pkg/kv/kvserver/gc/gc_iterator_test.go @@ -35,14 +35,14 @@ func TestGCIterator(t *testing.T) { txn *roachpb.Transaction } // makeDataItem is a shorthand to construct dataItems. - makeDataItem := func(k roachpb.Key, val []byte, ts int64, txn *roachpb.Transaction) dataItem { + makeDataItem := func(k roachpb.Key, val roachpb.Value, ts int64, txn *roachpb.Transaction) dataItem { return dataItem{ MVCCKeyValue: storage.MVCCKeyValue{ Key: storage.MVCCKey{ Key: k, Timestamp: hlc.Timestamp{WallTime: ts * time.Nanosecond.Nanoseconds()}, }, - Value: val, + Value: val.RawBytes, }, txn: txn, } diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan.go b/pkg/kv/kvserver/rangefeed/catchup_scan.go index 2f30a6d04087..357d4c63cfb7 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan.go @@ -157,10 +157,11 @@ func (i *CatchUpIterator) CatchUpScan( } unsafeKey := i.UnsafeKey() - unsafeVal := i.UnsafeValue() + unsafeValRaw := i.UnsafeValue() + var unsafeVal []byte if !unsafeKey.IsValue() { // Found a metadata key. - if err := protoutil.Unmarshal(unsafeVal, &meta); err != nil { + if err := protoutil.Unmarshal(unsafeValRaw, &meta); err != nil { return errors.Wrapf(err, "unmarshaling mvcc meta: %v", unsafeKey) } if !meta.IsInline() { @@ -195,6 +196,12 @@ func (i *CatchUpIterator) CatchUpScan( // iterators may result in the rangefeed not // seeing some intermediate values. unsafeVal = meta.RawBytes + } else { + mvccVal, err := storage.DecodeMVCCValue(unsafeValRaw) + if err != nil { + return errors.Wrapf(err, "decoding mvcc value: %v", unsafeKey) + } + unsafeVal = mvccVal.Value.RawBytes } // Ignore the version if it's not inline and its timestamp is at diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go index 184b509aa6ee..7991581958a1 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go @@ -34,10 +34,10 @@ func TestCatchupScan(t *testing.T) { testKey1 = roachpb.Key("/db1") testKey2 = roachpb.Key("/db2") - testValue1 = []byte("val1") - testValue2 = []byte("val2") - testValue3 = []byte("val3") - testValue4 = []byte("val4") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") + testValue3 = roachpb.MakeValueFromString("val3") + testValue4 = roachpb.MakeValueFromString("val4") ts1 = hlc.Timestamp{WallTime: 1, Logical: 0} ts2 = hlc.Timestamp{WallTime: 2, Logical: 0} @@ -46,7 +46,7 @@ func TestCatchupScan(t *testing.T) { ts5 = hlc.Timestamp{WallTime: 4, Logical: 0} ) - makeTxn := func(key roachpb.Key, val []byte, ts hlc.Timestamp, + makeTxn := func(key roachpb.Key, val roachpb.Value, ts hlc.Timestamp, ) (roachpb.Transaction, roachpb.Value) { txnID := uuid.MakeV4() txnMeta := enginepb.TxnMeta{ @@ -59,12 +59,12 @@ func TestCatchupScan(t *testing.T) { TxnMeta: txnMeta, ReadTimestamp: ts, }, roachpb.Value{ - RawBytes: val, + RawBytes: val.RawBytes, } } - makeKTV := func(key roachpb.Key, ts hlc.Timestamp, value []byte) storage.MVCCKeyValue { - return storage.MVCCKeyValue{Key: storage.MVCCKey{Key: key, Timestamp: ts}, Value: value} + makeKTV := func(key roachpb.Key, ts hlc.Timestamp, value roachpb.Value) storage.MVCCKeyValue { + return storage.MVCCKeyValue{Key: storage.MVCCKey{Key: key, Timestamp: ts}, Value: value.RawBytes} } // testKey1 has an intent and provisional value that will be skipped. Both // testKey1 and testKey2 have a value that is older than what we need with diff --git a/pkg/kv/kvserver/rangefeed/registry_test.go b/pkg/kv/kvserver/rangefeed/registry_test.go index 315ed7746710..a2cd8f243f11 100644 --- a/pkg/kv/kvserver/rangefeed/registry_test.go +++ b/pkg/kv/kvserver/rangefeed/registry_test.go @@ -286,50 +286,50 @@ func TestRegistrationCatchUpScan(t *testing.T) { expEvents := []*roachpb.RangeFeedEvent{ rangeFeedValueWithPrev( roachpb.Key("d"), - roachpb.Value{RawBytes: []byte("valD3"), Timestamp: hlc.Timestamp{WallTime: 16}}, - roachpb.Value{RawBytes: []byte("valD2")}, + makeValWithTs("valD3", 16), + makeVal("valD2"), ), rangeFeedValueWithPrev( roachpb.Key("d"), - roachpb.Value{RawBytes: []byte("valD4"), Timestamp: hlc.Timestamp{WallTime: 19}}, - roachpb.Value{RawBytes: []byte("valD3")}, + makeValWithTs("valD4", 19), + makeVal("valD3"), ), rangeFeedValueWithPrev( roachpb.Key("d"), - roachpb.Value{RawBytes: []byte("valD5"), Timestamp: hlc.Timestamp{WallTime: 20}}, - roachpb.Value{RawBytes: []byte("valD4")}, + makeValWithTs("valD5", 20), + makeVal("valD4"), ), rangeFeedValueWithPrev( roachpb.Key("e"), - roachpb.Value{RawBytes: []byte("valE2"), Timestamp: hlc.Timestamp{WallTime: 5}}, - roachpb.Value{RawBytes: []byte("valE1")}, + makeValWithTs("valE2", 5), + makeVal("valE1"), ), rangeFeedValueWithPrev( roachpb.Key("e"), - roachpb.Value{RawBytes: []byte("valE3"), Timestamp: hlc.Timestamp{WallTime: 6}}, - roachpb.Value{RawBytes: []byte("valE2")}, + makeValWithTs("valE3", 6), + makeVal("valE2"), ), rangeFeedValue( roachpb.Key("f"), - roachpb.Value{RawBytes: []byte("valF1"), Timestamp: hlc.Timestamp{WallTime: 5}}, + makeValWithTs("valF1", 5), ), rangeFeedValueWithPrev( roachpb.Key("f"), - roachpb.Value{RawBytes: []byte("valF2"), Timestamp: hlc.Timestamp{WallTime: 6}}, - roachpb.Value{RawBytes: []byte("valF1")}, + makeValWithTs("valF2", 6), + makeVal("valF1"), ), rangeFeedValueWithPrev( roachpb.Key("f"), - roachpb.Value{RawBytes: []byte("valF3"), Timestamp: hlc.Timestamp{WallTime: 7}}, - roachpb.Value{RawBytes: []byte("valF2")}, + makeValWithTs("valF3", 7), + makeVal("valF2"), ), rangeFeedValue( roachpb.Key("g"), - roachpb.Value{RawBytes: []byte("valG1"), Timestamp: hlc.Timestamp{WallTime: 0}}, + makeVal("valG1"), ), rangeFeedValue( roachpb.Key("h"), - roachpb.Value{RawBytes: []byte("valH1"), Timestamp: hlc.Timestamp{WallTime: 15}}, + makeValWithTs("valH1", 15), ), } require.Equal(t, expEvents, r.Events()) diff --git a/pkg/kv/kvserver/rangefeed/task_test.go b/pkg/kv/kvserver/rangefeed/task_test.go index 9c87c4f93e40..45971de7c0c3 100644 --- a/pkg/kv/kvserver/rangefeed/task_test.go +++ b/pkg/kv/kvserver/rangefeed/task_test.go @@ -27,13 +27,23 @@ import ( "github.com/stretchr/testify/require" ) +func makeVal(val string) roachpb.Value { + return roachpb.MakeValueFromString(val) +} + +func makeValWithTs(val string, ts int64) roachpb.Value { + v := makeVal(val) + v.Timestamp = hlc.Timestamp{WallTime: ts} + return v +} + func makeKV(key, val string, ts int64) storage.MVCCKeyValue { return storage.MVCCKeyValue{ Key: storage.MVCCKey{ Key: roachpb.Key(key), Timestamp: hlc.Timestamp{WallTime: ts}, }, - Value: []byte(val), + Value: makeVal(val).RawBytes, } } @@ -56,7 +66,7 @@ func makeMetaKV(key string, meta enginepb.MVCCMetadata) storage.MVCCKeyValue { func makeInline(key, val string) storage.MVCCKeyValue { return makeMetaKV(key, enginepb.MVCCMetadata{ - RawBytes: []byte(val), + RawBytes: makeVal(val).RawBytes, }) } diff --git a/pkg/kv/kvserver/replica_rangefeed_test.go b/pkg/kv/kvserver/replica_rangefeed_test.go index 89b58da9f1c8..9a6960c99b94 100644 --- a/pkg/kv/kvserver/replica_rangefeed_test.go +++ b/pkg/kv/kvserver/replica_rangefeed_test.go @@ -341,10 +341,10 @@ func TestReplicaRangefeed(t *testing.T) { defer sstWriter.Close() require.NoError(t, sstWriter.PutMVCC( storage.MVCCKey{Key: roachpb.Key("b"), Timestamp: ts6}, - expVal6b.RawBytes)) + storage.MVCCValue{Value: expVal6b})) require.NoError(t, sstWriter.PutMVCC( storage.MVCCKey{Key: roachpb.Key("q"), Timestamp: ts6}, - expVal6q.RawBytes)) + storage.MVCCValue{Value: expVal6q})) require.NoError(t, sstWriter.Finish()) expSST := sstFile.Data() expSSTSpan := roachpb.Span{Key: roachpb.Key("b"), EndKey: roachpb.Key("r")} @@ -357,11 +357,11 @@ func TestReplicaRangefeed(t *testing.T) { // Ingest an SSTable as writes. ts7 := ts.Clock().Now().Add(0, 7) - expVal7b := roachpb.Value{Timestamp: ts7} + expVal7b := roachpb.Value{} expVal7b.SetInt(7) expVal7b.InitChecksum(roachpb.Key("b")) - expVal7q := roachpb.Value{Timestamp: ts7} + expVal7q := roachpb.Value{} expVal7q.SetInt(7) expVal7q.InitChecksum(roachpb.Key("q")) @@ -370,10 +370,10 @@ func TestReplicaRangefeed(t *testing.T) { defer sstWriter.Close() require.NoError(t, sstWriter.PutMVCC( storage.MVCCKey{Key: roachpb.Key("b"), Timestamp: ts7}, - expVal7b.RawBytes)) + storage.MVCCValue{Value: expVal7b})) require.NoError(t, sstWriter.PutMVCC( storage.MVCCKey{Key: roachpb.Key("q"), Timestamp: ts7}, - expVal7q.RawBytes)) + storage.MVCCValue{Value: expVal7q})) require.NoError(t, sstWriter.Finish()) _, _, _, pErr = store1.DB().AddSSTableAtBatchTimestamp(ctx, roachpb.Key("b"), roachpb.Key("r"), sstFile.Data(), @@ -391,6 +391,8 @@ func TestReplicaRangefeed(t *testing.T) { expVal5 := roachpb.Value{Timestamp: ts5} expVal5.SetInt(25) expVal5.InitChecksum(roachpb.Key("b")) + expVal7b.Timestamp = ts7 + expVal7q.Timestamp = ts7 expVal1NoTS, expVal4NoTS := expVal1, expVal4 expVal1NoTS.Timestamp, expVal4NoTS.Timestamp = hlc.Timestamp{}, hlc.Timestamp{} expEvents = append(expEvents, []*roachpb.RangeFeedEvent{ diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index ae012cb87a87..a8ac49d22083 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -612,13 +612,20 @@ func (s spanSetWriter) Merge(key storage.MVCCKey, value []byte) error { return s.w.Merge(key, value) } -func (s spanSetWriter) PutMVCC(key storage.MVCCKey, value []byte) error { +func (s spanSetWriter) PutMVCC(key storage.MVCCKey, value storage.MVCCValue) error { if err := s.checkAllowed(key.Key); err != nil { return err } return s.w.PutMVCC(key, value) } +func (s spanSetWriter) PutRawMVCC(key storage.MVCCKey, value []byte) error { + if err := s.checkAllowed(key.Key); err != nil { + return err + } + return s.w.PutRawMVCC(key, value) +} + func (s spanSetWriter) PutUnversioned(key roachpb.Key, value []byte) error { if err := s.checkAllowed(key); err != nil { return err diff --git a/pkg/kv/kvserver/uncertainty/doc.go b/pkg/kv/kvserver/uncertainty/doc.go index 81fc80dc6941..c4b431b871c5 100644 --- a/pkg/kv/kvserver/uncertainty/doc.go +++ b/pkg/kv/kvserver/uncertainty/doc.go @@ -20,10 +20,69 @@ // uncertainty related restarts. package uncertainty -import "github.com/cockroachdb/cockroach/pkg/roachpb" +import ( + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" +) + +// D0 ———————————————————————————————————————————————— +// +// MVCCKey +// +// Each value in CockroachDB is stored at an associated versioned key. This key +// is distinguished from roachpb.Key with the addition of a "version" timestamp. +// +// As a multi-version concurrency control (MVCC) system, the version timestamp +// dictates the key's visibility to readers. Readers with read timestamps equal +// to or greater than the version timestamp observe the key. Readers with read +// timestamps below the version timestamp ignore the key. +// +// NOTE: inlined struct definition to avoid circular dependency. +var D0 = /* storage.MVCCKey */ struct { + Key roachpb.Key + Timestamp hlc.Timestamp +}{} // D1 ———————————————————————————————————————————————— // +// MVCCValue +// +// At and below the MVCC layer, each roachpb.Value as wrapped with a header that +// contains MVCC-level metadata. One piece of metadata stored in this header is +// the value's "local" clock timestamp. +// +// The local clock timestamp records the value of the local HLC clock on the +// leaseholder when the key-value was originally written. It is used to make +// claims about the relative real time ordering of the key-value's writer and +// readers. +// +// When a versioned key-value is encountered by a transaction while reading, +// there are three possible outcomes: +// 1. the key's value is visible to the reader and is returned. +// 2. the key's value is not visible to the reader but is determined to have +// definitely been written concurrently with reader's transaction, so it is +// ignored and not returned. +// 3. the key's value is not visible to the reader but is determined to have +// been possibly written before the reader's transaction, so the reader must +// perform an uncertainty restart and later return the value to ensure real- +// time ordering guarantees. +// +// We call the differentiation between outcomes 2 and 3 "uncertainty". The rest +// of this package explores the mechanisms used to guarantee real-time ordering +// between transactions while attempting to maximize the number of readers that +// end up in outcome 2 and minimize the number that end up in outcome 3, thereby +// minimizing uncertainty restarts and maximizing concurrency. +// +// NOTE: inlined struct definition to avoid circular dependency. +var D1 = /* storage.MVCCValue */ struct { + Header struct { + LocalTimestamp hlc.ClockTimestamp + } + Value roachpb.Value +}{} + +// D2 ———————————————————————————————————————————————— +// // Transaction.GlobalUncertaintyLimit // // A transaction's global uncertainty limit is the inclusive upper bound of its @@ -32,9 +91,9 @@ import "github.com/cockroachdb/cockroach/pkg/roachpb" // respected, this maximum timestamp places an upper bound on the commit // timestamp of any other transaction that committed causally before the new // transaction. -var D1 = roachpb.Transaction{}.GlobalUncertaintyLimit +var D2 = roachpb.Transaction{}.GlobalUncertaintyLimit -// D2 ———————————————————————————————————————————————— +// D3 ———————————————————————————————————————————————— // // Interval // @@ -43,9 +102,9 @@ var D1 = roachpb.Transaction{}.GlobalUncertaintyLimit // global uncertainty limit is pulled directly from the transaction. The local // uncertainty limit is an optional tighter bound established through HLC clock // observations on individual nodes in the system. -var D2 = Interval{} +var D3 = Interval{} -// D3 ———————————————————————————————————————————————— +// D4 ———————————————————————————————————————————————— // // ReadWithinUncertaintyIntervalError // @@ -59,9 +118,9 @@ var D2 = Interval{} // guaranteed to observe any value written by any other transaction with a // happened-before relation to it, which is paramount to ensure single-key // linearizability and avoid stale reads. -var D3 = roachpb.ReadWithinUncertaintyIntervalError{} +var D4 = roachpb.ReadWithinUncertaintyIntervalError{} -// D4 ———————————————————————————————————————————————— +// D5 ———————————————————————————————————————————————— // // ObservedTimestamp // @@ -73,17 +132,17 @@ var D3 = roachpb.ReadWithinUncertaintyIntervalError{} // promise that any later clock reading from the same node will have a larger // value. Similarly, they are a promise that any prior update to the clock was // given a smaller value. -var D4 = roachpb.ObservedTimestamp{} +var D5 = roachpb.ObservedTimestamp{} -// D5 ———————————————————————————————————————————————— +// D6 ———————————————————————————————————————————————— // // Transaction.UpdateObservedTimestamp // // A transaction collects observed timestamps as it visits nodes in the cluster // when performing reads and writes. -var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp +var D6 = (&roachpb.Transaction{}).UpdateObservedTimestamp -// D6 ———————————————————————————————————————————————— +// D7 ———————————————————————————————————————————————— // // Transaction.ObservedTimestamps // @@ -91,7 +150,10 @@ var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp // purpose of this list is to avoid uncertainty related restarts which occur // when reading a value in the near future, per the global_uncertainty_limit // field. The list helps avoid these restarts by establishing a lower -// local_uncertainty_limit when evaluating a request on a node in the list. +// local_uncertainty_limit when evaluating a request on a node in the list. The +// local_uncertainty_limit can then be compared against the value's local +// timestamp to determine whether the value was written before or after the +// clock observation by the reader. // // Meaning // @@ -115,29 +177,32 @@ var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp // // Thus, expressed properly, we can say that when a node has been read from // successfully before by a transaction, uncertainty for values written by a -// leaseholder on that node is restricted to values with timestamps in the +// leaseholder on that node is restricted to values with local timestamps in the // interval [read_timestamp, first_visit_timestamp). An upper bound can be // placed on the uncertainty window because we are guaranteed that at the time // that the transaction first visited the node, none of the Ranges that it was a -// leaseholder for had served any writes at higher timestamps than the clock -// reading we observe. This implies the following property: +// leaseholder for had assigned local timestamps to any writes at higher +// timestamps than the clock reading we observe. This implies the following +// property: // // Any writes that the transaction may later see written by leaseholders on -// this node at higher timestamps than the observed timestamp could not have -// taken place causally before this transaction and can be ignored for the -// purposes of uncertainty. +// this node at higher local timestamps than the observed timestamp could not +// have taken place causally before this transaction and can be ignored for +// the purposes of uncertainty. // // There are two invariants necessary for this property to hold: // -// 1. a leaseholder's clock must always be equal to or greater than the timestamp -// of all writes that it has served. This is trivial to enforce for +// 1. a leaseholder's clock must always be equal to or greater than the local +// timestamp of all writes that it has served. This is trivial to enforce for // non-transactional writes. It is more complicated for transactional writes // which may move their commit timestamp forward over their lifetime before // committing, even after writing intents on remote Ranges. To accommodate -// this situation, transactions ensure that at the time of their commit, any -// leaseholder for a Range that contains one of its intent has an HLC clock -// with an equal or greater timestamp than the transaction's commit timestamp. -// TODO(nvanbenschoten): This is violated by txn refreshes. See #36431. +// this situation, MVCC key-values track their local timestamp separately +// from their version timestamp. This provides the flexibility to advance a +// key-value's version timestamp without advancing its local timestamp and +// ensure that at the time of a transaction's commit, any leaseholder for a +// Range that contains one of its intent has an HLC clock with a greater +// timestamp than that intent's local timestamp. // // 2. a leaseholder's clock must always be equal to or greater than the timestamp // of all writes that previous leaseholders for its Range have served. We @@ -153,14 +218,17 @@ var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp // // The property ensures that when this list holds a corresponding entry for the // node who owns the lease that the current request is executing under, we can -// run the request with the list's timestamp as the upper bound for its -// uncertainty interval, limiting (and often avoiding) uncertainty restarts. We -// do this by establishing a separate local_uncertainty_limit, which is set to -// the minimum of the global_uncertainty_limit and the node's observed timestamp -// entry in ComputeLocalUncertaintyLimit. +// compare the list's timestamp against the local timestamp of values in the +// range. This comparison establishes a total ordering across clock observations +// (local timestamp assignment for writers and observed timestamps for readers) +// and establish a partial ordering between writer and reader transactions. This +// partial ordering can limit (and often avoid) uncertainty restarts. We do this +// by establishing a separate local_uncertainty_limit for readers, which is set +// to the minimum of the global_uncertainty_limit and the node's observed +// timestamp entry in ComputeLocalUncertaintyLimit. // // However, as stated, the correctness property only holds for values at higher -// timestamps than the observed timestamp written *by leaseholders on this +// local timestamps than the observed timestamp written *by leaseholders on this // node*. This is critical, as the property tells us nothing about values // written by leaseholders on different nodes, even if a lease for one of those // Ranges has since moved to a node that we have an observed timestamp entry @@ -182,17 +250,18 @@ var D5 = (&roachpb.Transaction{}).UpdateObservedTimestamp // meaning for the purpose of reducing the transaction's uncertainty interval. // This is because there is no guarantee that at the time of acquiring the // observed timestamp from the follower node, the leaseholder hadn't already -// served writes at higher timestamps than the follower node's clock reflected. +// served writes with local timestamps higher than the follower node's clock +// reflected. // // However, if the transaction performing a follower read happens to have an // observed timestamp from the current leaseholder, this timestamp can be used // to reduce the transaction's uncertainty interval. Even though the read is // being served from a different replica in the range, the observed timestamp -// still places a bound on the values in the range that may have been written -// before the transaction began. -var D6 = roachpb.Transaction{}.ObservedTimestamps +// still places a bound on the local timestamps associated with values in the +// range that may have been written before the transaction began. +var D7 = roachpb.Transaction{}.ObservedTimestamps -// D7 ———————————————————————————————————————————————— +// D8 ———————————————————————————————————————————————— // // TimestampFromServerClock // @@ -219,7 +288,7 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // receive an uncertainty error. // // It is somewhat non-intuitive that non-transactional requests need uncertainty -// intervals — after all, they receive their timestamp to the leaseholder of the +// intervals — after all, they receive their timestamp on the leaseholder of the // only range that they talk to, so isn't every value with a commit timestamp // above their read timestamp certainly concurrent? The answer is surprisingly // "no" for the following reasons, so they cannot forgo the use of uncertainty @@ -229,7 +298,7 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // This means that there are times when the replica is not the leaseholder at // the point of timestamp allocation, and only becomes the leaseholder later. // In such cases, the timestamp assigned to the request is not guaranteed to -// be greater than the written_timestamp of all writes served by the range at +// be greater than the local timestamp of all writes served by the range at // the time of allocation. This is true despite invariants 1 & 2 from above, // because the replica allocating the timestamp is not yet the leaseholder. // @@ -253,7 +322,7 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // // 2. even if the replica's lease is stable and the timestamp is assigned to the // non-transactional request by the leaseholder, the assigned clock reading -// only reflects the written_timestamp of all of the writes served by the +// only reflects the local timestamp of all of the writes served by the // leaseholder (and previous leaseholders) thus far. This clock reading is // not guaranteed to lead the commit timestamp of all of these writes, // especially if they are committed remotely and resolved after the request @@ -266,21 +335,21 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // the leaseholder to be uncertain. Concretely, the non-transactional request // needs to consider values of the following form to be uncertain: // -// written_timestamp < local_limit && commit_timestamp < global_limit +// local_timestamp < local_limit && commit_timestamp < global_limit // // The value that the non-transactional request is observing may have been // written on the local leaseholder at time 10, its transaction may have been // committed remotely at time 20, acknowledged, then the non-transactional // request may have begun and received a timestamp of 15 from the local // leaseholder, then finally the value may have been resolved asynchronously -// and moved to timestamp 20 (written_timestamp: 10, commit_timestamp: 20). -// The failure of the non-transactional request to observe this value would -// be a stale read. +// and moved to timestamp 20 (local_timestamp: 10, commit_timestamp: 20). The +// failure of the non-transactional request to observe this value would be a +// stale read. // // For example, consider the following series of events: // - client begins a txn and is assigned provisional commit timestamp = 10 // - client's txn performs a Put(k, v1) -// - leaseholder serves Put(k, v1), lays down intent at written_timestamp = 10 +// - leaseholder serves Put(k, v1), lays down intent at local_timestamp = 10 // - client's txn performs a write elsewhere and hits a WriteTooOldError // that bumps its provisional commit timestamp to 20 // - client's txn refreshes to ts = 20. This notably happens without @@ -296,22 +365,15 @@ var D6 = roachpb.Transaction{}.ObservedTimestamps // server-side retry, bumps its read timestamp, and returns k = v1. Failure // to do so would be a stale read. // -// TODO(nvanbenschoten): expand on this when we fix #36431. For now, this can -// be framed in relation to synthetic timestamps, but it's easier to discuss -// in terms of the impending "written_timestamp" attribute of each value, -// even though written_timestamps do not yet exist in code. -// -// TODO(nvanbenschoten): add more direct testing for this when we fix #36431. -// // TODO(nvanbenschoten): add another reason here once we address #73292. // // Convenient, because non-transactional requests are always scoped to a // single-range, those that hit uncertainty errors can always retry on the // server, so these errors never bubble up to the client that initiated the // request. -var D7 = roachpb.Header{}.TimestampFromServerClock +var D8 = roachpb.Header{}.TimestampFromServerClock -// D8 ———————————————————————————————————————————————— +// D9 ———————————————————————————————————————————————— // // ComputeInterval // @@ -321,7 +383,7 @@ var D7 = roachpb.Header{}.TimestampFromServerClock // Similarly, observed timestamps can also assist a transaction even on its // first visit to a node in cases where it gets stuck waiting on locks for long // periods of time. -var D8 = ComputeInterval +var D9 = ComputeInterval // Ignore unused warnings. -var _, _, _, _, _, _, _, _ = D1, D2, D3, D4, D5, D6, D7, D8 +var _, _, _, _, _, _, _, _, _, _ = D0, D1, D2, D3, D4, D5, D6, D7, D8, D9 diff --git a/pkg/kv/kvserver/uncertainty/interval.go b/pkg/kv/kvserver/uncertainty/interval.go index be9de5d8e4f3..22c4e5520f78 100644 --- a/pkg/kv/kvserver/uncertainty/interval.go +++ b/pkg/kv/kvserver/uncertainty/interval.go @@ -31,11 +31,9 @@ import "github.com/cockroachdb/cockroach/pkg/util/hlc" // a range. This can lead to values that would otherwise be considered uncertain // by the original global limit to be considered "certainly concurrent", and // thus not causally related, with the transaction due to observed timestamps. -// -// However, the local limit does not apply to all committed values on a range. -// Specifically, values with "synthetic timestamps" must use the interval's -// global limit for the purposes of uncertainty, because observed timestamps do -// not apply to values with synthetic timestamps. +// However, the local limit does not operate on a value's version timestamp. It +// instead applies to a value's local timestamp, which is a recording of the +// local HLC clock on the leaseholder that originally wrote the value. // // Uncertainty intervals also apply to non-transactional requests that require // strong consistency (single-key linearizability). These requests defer their @@ -50,12 +48,15 @@ type Interval struct { LocalLimit hlc.ClockTimestamp } -// IsUncertain determines whether a value with the provided timestamp is -// uncertain to a reader with a ReadTimestamp below the value's and with -// the specified uncertainty interval. -func (in Interval) IsUncertain(valueTs hlc.Timestamp) bool { - if !in.LocalLimit.IsEmpty() && !valueTs.Synthetic { - return valueTs.LessEq(in.LocalLimit.ToTimestamp()) +// IsUncertain determines whether a value with the provided version and local +// timestamps is uncertain to a reader with a ReadTimestamp below the value's +// version timestamp and with the specified uncertainty interval. +func (in Interval) IsUncertain(valueTs hlc.Timestamp, localTs hlc.ClockTimestamp) bool { + if !in.LocalLimit.IsEmpty() && in.LocalLimit.Less(localTs) { + // The reader has an observed timestamp that precedes the local timestamp of + // this value. There is no uncertainty as the reader transaction must have + // started before the writer transaction completed, so they are concurrent. + return false } return valueTs.LessEq(in.GlobalLimit) } diff --git a/pkg/kv/kvserver/uncertainty/interval_test.go b/pkg/kv/kvserver/uncertainty/interval_test.go index 028640128944..065e2789522e 100644 --- a/pkg/kv/kvserver/uncertainty/interval_test.go +++ b/pkg/kv/kvserver/uncertainty/interval_test.go @@ -24,53 +24,83 @@ func TestInterval_IsUncertain(t *testing.T) { makeTs := func(walltime int64) hlc.Timestamp { return hlc.Timestamp{WallTime: walltime} } - makeSynTs := func(walltime int64) hlc.Timestamp { - return makeTs(walltime).WithSynthetic(true) - } emptyTs := makeTs(0) testCases := []struct { - localLim, globalLim, valueTs hlc.Timestamp - exp bool + localLim, globalLim, valueTs, localTs hlc.Timestamp + exp bool }{ - // Without synthetic value. - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(5), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(10), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), exp: false}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(5), exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(10), exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), exp: false}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), exp: false}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), exp: false}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(5), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(10), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), exp: false}, - // With synthetic value. - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeSynTs(5), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeSynTs(10), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeSynTs(15), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeSynTs(20), exp: true}, - {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeSynTs(25), exp: false}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeSynTs(5), exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeSynTs(10), exp: true}, - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeSynTs(15), exp: true}, // different - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeSynTs(20), exp: true}, // different - {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeSynTs(25), exp: false}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeSynTs(5), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeSynTs(10), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeSynTs(15), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeSynTs(20), exp: true}, - {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeSynTs(25), exp: false}, + // Without local timestamp. localLim is ignored. + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(5), localTs: emptyTs, exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(10), localTs: emptyTs, exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), localTs: emptyTs, exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: emptyTs, exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: emptyTs, exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(5), localTs: emptyTs, exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(10), localTs: emptyTs, exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), localTs: emptyTs, exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: emptyTs, exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: emptyTs, exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(5), localTs: emptyTs, exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(10), localTs: emptyTs, exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), localTs: emptyTs, exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: emptyTs, exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: emptyTs, exp: false}, + // With local timestamp equal to value timestamp. + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(5), localTs: makeTs(5), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(10), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(15), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(20), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(25), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(5), localTs: makeTs(5), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(10), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(15), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(20), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(25), exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(5), localTs: makeTs(5), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(10), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(15), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(20), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(25), exp: false}, + // With local timestamp below value timestamp. + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(5), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(5), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(10), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(5), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(10), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(15), exp: true}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(5), exp: false}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(10), exp: false}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(15), exp: false}, + {localLim: emptyTs, globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(20), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(5), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(5), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(10), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(5), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(10), exp: true}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(15), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(5), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(10), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(15), exp: false}, + {localLim: makeTs(10), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(20), exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(10), localTs: makeTs(5), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(5), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(15), localTs: makeTs(10), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(5), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(10), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(20), localTs: makeTs(15), exp: true}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(5), exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(10), exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(15), exp: false}, + {localLim: makeTs(20), globalLim: makeTs(20), valueTs: makeTs(25), localTs: makeTs(20), exp: false}, // Empty uncertainty intervals. - {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(5), exp: false}, - {localLim: emptyTs, globalLim: emptyTs, valueTs: makeSynTs(5), exp: false}, + {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(10), localTs: emptyTs, exp: false}, + {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(10), localTs: makeTs(10), exp: false}, + {localLim: emptyTs, globalLim: emptyTs, valueTs: makeTs(10), localTs: makeTs(5), exp: false}, } for _, test := range testCases { in := Interval{GlobalLimit: test.globalLim, LocalLimit: hlc.ClockTimestamp(test.localLim)} - require.Equal(t, test.exp, in.IsUncertain(test.valueTs), "%+v", test) + res := in.IsUncertain(test.valueTs, hlc.ClockTimestamp(test.localTs)) + require.Equal(t, test.exp, res, "%+v", test) } } diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index fc6b94dafd37..e6814bd9b86d 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -929,6 +929,7 @@ func MakeTransaction( // occurred, i.e. the maximum of ReadTimestamp and LastHeartbeat. func (t Transaction) LastActive() hlc.Timestamp { ts := t.LastHeartbeat + // TODO(nvanbenschoten): remove this when we remove synthetic timestamps. if !t.ReadTimestamp.Synthetic { ts.Forward(t.ReadTimestamp) } diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index f838acca0e97..46d84c2acf74 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -67,6 +67,12 @@ enum ValueType { // TIMESERIES is applied to values which contain InternalTimeSeriesData. TIMESERIES = 100; + + // MVCC_EXTENDED_ENCODING_SENTINEL is a sentinel tag value used in the + // MVCCValue extended encoding to indicate that an encoded Value is wrapped + // with a header. It should not be used in the Value encoding directly. See + // MVCCValue for more details. + MVCC_EXTENDED_ENCODING_SENTINEL = 101; } // Value specifies the value at a key. Multiple values at the same key are diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index a5312ed09203..640e91221fd4 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -934,6 +934,9 @@ var _ transactionRestartError = &WriteTooOldError{} // NewReadWithinUncertaintyIntervalError creates a new uncertainty retry error. // The read and existing timestamps as well as the txn are purely informational // and used for formatting the error message. +// TODO(nvanbenschoten): change localUncertaintyLimit to hlc.ClockTimestamp. +// TODO(nvanbenschoten): change existingTS to versionTs. +// TODO(nvanbenschoten): add localTs and include in error string. func NewReadWithinUncertaintyIntervalError( readTS, existingTS, localUncertaintyLimit hlc.Timestamp, txn *Transaction, ) *ReadWithinUncertaintyIntervalError { diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index 520d26de582f..385821fd2f20 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -1169,7 +1169,10 @@ func TestAssertEnginesEmpty(t *testing.T) { Key: []byte{0xde, 0xad, 0xbe, 0xef}, Timestamp: hlc.Timestamp{WallTime: 100}, } - require.NoError(t, batch.PutMVCC(key, []byte("foo"))) + value := storage.MVCCValue{ + Value: roachpb.MakeValueFromString("foo"), + } + require.NoError(t, batch.PutMVCC(key, value)) require.NoError(t, batch.Commit(false)) require.Error(t, assertEnginesEmpty([]storage.Engine{eng})) } diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index a92b50a04867..af04e1c3b7b6 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -21,6 +21,7 @@ go_library( "mvcc_incremental_iterator.go", "mvcc_key.go", "mvcc_logical_ops.go", + "mvcc_value.go", "open.go", "pebble.go", "pebble_batch.go", @@ -110,6 +111,7 @@ go_test( "mvcc_logical_ops_test.go", "mvcc_stats_test.go", "mvcc_test.go", + "mvcc_value_test.go", "pebble_file_registry_test.go", "pebble_mvcc_scanner_test.go", "pebble_test.go", diff --git a/pkg/storage/batch_test.go b/pkg/storage/batch_test.go index 43f9cff57c39..495bc83628cb 100644 --- a/pkg/storage/batch_test.go +++ b/pkg/storage/batch_test.go @@ -1120,7 +1120,7 @@ func TestDecodeKey(t *testing.T) { t.Fatalf("%+v", err) } } else { - if err := b.PutMVCC(test, nil); err != nil { + if err := b.PutMVCC(test, MVCCValue{}); err != nil { t.Fatalf("%+v", err) } } diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index 7c848a84dd8b..be4628e9c2a7 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -190,7 +190,6 @@ func setupKeysWithIntent( txnIDWithLatestVersion := adjustTxnID(numVersions) otherTxnWithLatestVersion := txnIDCount + 2 otherTxnUUID := uuid.FromUint128(uint128.FromInts(0, uint64(otherTxnWithLatestVersion))) - val := []byte("value") var rvLockUpdate roachpb.LockUpdate for i := 1; i <= numVersions; i++ { // Assign txn IDs in a deterministic way that will mimic the end result of @@ -236,7 +235,7 @@ func setupKeysWithIntent( } } } - value := roachpb.Value{RawBytes: val} + value := roachpb.MakeValueFromString("value") batch := eng.NewBatch() for j := 0; j < numIntentKeys; j++ { putTxn := &txn @@ -1521,7 +1520,9 @@ func runExportToSst( key = encoding.EncodeUint32Ascending(key, uint32(i)) for j := 0; j < numRevisions; j++ { - err := batch.PutMVCC(MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(j + 1), Logical: 0}}, []byte("foobar")) + mvccKey := MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(j + 1), Logical: 0}} + mvccValue := MVCCValue{Value: roachpb.MakeValueFromString("foobar")} + err := batch.PutMVCC(mvccKey, mvccValue) if err != nil { b.Fatal(err) } @@ -1564,10 +1565,11 @@ func (noopWriter) Write(p []byte) (int, error) { return len(p), nil } func runCheckSSTConflicts(b *testing.B, numEngineKeys, numVersions, numSstKeys int, overlap bool) { keyBuf := append(make([]byte, 0, 64), []byte("key-")...) - value := make([]byte, 128) - for i := range value { - value[i] = 'a' + valueBuf := make([]byte, 128) + for i := range valueBuf { + valueBuf[i] = 'a' } + value := MVCCValue{Value: roachpb.MakeValueFromBytes(valueBuf)} eng := setupMVCCInMemPebble(b, "") defer eng.Close() @@ -1577,7 +1579,7 @@ func runCheckSSTConflicts(b *testing.B, numEngineKeys, numVersions, numSstKeys i for j := 0; j < numVersions; j++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) ts := hlc.Timestamp{WallTime: int64(j + 1)} - require.NoError(b, batch.PutMVCC(MVCCKey{key, ts}, value)) + require.NoError(b, batch.PutMVCC(MVCCKey{Key: key, Timestamp: ts}, value)) } require.NoError(b, batch.Commit(false)) } @@ -1605,7 +1607,7 @@ func runCheckSSTConflicts(b *testing.B, numEngineKeys, numVersions, numSstKeys i sstEnd.Key = append([]byte(nil), mvccKey.Key...) sstEnd.Timestamp = mvccKey.Timestamp } - require.NoError(b, sstWriter.Put(mvccKey, value)) + require.NoError(b, sstWriter.PutMVCC(mvccKey, value)) } sstWriter.Close() diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 7b86246eaca2..fa52de4a8f8d 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -589,8 +589,15 @@ type Writer interface { // timestamp is non-empty (see {PutUnversioned,PutIntent} if the timestamp // is empty). // - // It is safe to modify the contents of the arguments after Put returns. - PutMVCC(key MVCCKey, value []byte) error + // It is safe to modify the contents of the arguments after PutMVCC returns. + PutMVCC(key MVCCKey, value MVCCValue) error + // PutRawMVCC is like PutMVCC, but it accepts an encoded MVCCValue. It + // can be used to avoid decoding and immediately re-encoding an MVCCValue, + // but should generally be avoided due to the lack of type safety. + // + // It is safe to modify the contents of the arguments after PutRawMVCC + // returns. + PutRawMVCC(key MVCCKey, value []byte) error // PutUnversioned sets the given key to the value provided. It is for use // with inline metadata (not intents) and other unversioned keys (like // Range-ID local keys). diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index cc8653983721..74949377dff9 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -611,7 +611,8 @@ func TestEngineTimeBound(t *testing.T) { for i, time := range times { s := fmt.Sprintf("%02d", i) key := MVCCKey{Key: roachpb.Key(s), Timestamp: time} - if err := engine.PutMVCC(key, []byte(s)); err != nil { + value := MVCCValue{Value: roachpb.MakeValueFromString(s)} + if err := engine.PutMVCC(key, value); err != nil { t.Fatal(err) } } diff --git a/pkg/storage/enginepb/mvcc.proto b/pkg/storage/enginepb/mvcc.proto index ffaf911e898e..6faa0c8bf355 100644 --- a/pkg/storage/enginepb/mvcc.proto +++ b/pkg/storage/enginepb/mvcc.proto @@ -58,7 +58,7 @@ message MVCCMetadata { // the IntentHistory. optional int32 sequence = 1 [(gogoproto.nullable) = false, (gogoproto.casttype) = "TxnSeq"]; // Value is the value written to the key as part of the transaction at - // the above Sequence. + // the above Sequence. Value uses the roachpb.Value encoding. optional bytes value = 2; } diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index a528febf764f..24147eac6cfe 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -140,6 +140,38 @@ message IgnoredSeqNumRange { int32 end = 2 [(gogoproto.casttype) = "TxnSeq"]; } +// MVCCValueHeader holds MVCC-level metadata for a versioned value. +// Used by storage.MVCCValue. +message MVCCValueHeader { + option (gogoproto.equal) = true; + + // The local clock timestamp records the value of the local HLC clock on the + // leaseholder when the key was originally written. It is used to make claims + // about the relative real time ordering of the key-value's writer and readers + // when comparing a reader's uncertainty interval (and observed timestamps) to + // the key-value. Ignoring edge cases, readers with an observed timestamp from + // the key-value's leaseholder that is greater than the local clock timestamp + // stored in the key cannot make claims about real time ordering and must + // consider it possible that the key-value's write occurred before the read + // began. However, readers with an observed timestamp from the key-value's + // leaseholder that is less than the clock timestamp can claim that the reader + // captured that observed timestamp before the key-value was written and + // therefore can consider the key-value's write to have been concurrent with + // the read. In doing so, the reader can avoid an uncertainty restart. For + // more, see pkg/kv/kvserver/observedts/doc.go. + // + // If the local timestamp is empty, it is assumed to be equal to the key's + // version timestamp. This property is used to avoid storing the local clock + // timestamp in the common case where the HLC clock on the leaseholder is + // greater than or equal to the version timestamp at the time that it is + // written. If is safe for the local clock timestamp to be rounded down, as + // this will simply lead to additional uncertainty restarts. However, it is + // not safe for the local clock timestamp to be rounded up, as this could lead + // to stale reads. + util.hlc.Timestamp local_timestamp = 1 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; +} + // MVCCStatsDelta is convertible to MVCCStats, but uses signed variable width // encodings for most fields that make it more efficient to store negative // values. This makes the encodings incompatible. diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go index cd586706b286..6d28cbe95e01 100644 --- a/pkg/storage/intent_interleaving_iter_test.go +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -168,8 +168,18 @@ func checkAndOutputIter(iter MVCCIterator, b *strings.Builder) { } return } + mvccVal, err := DecodeMVCCValue(v1) + if err != nil { + fmt.Fprintf(b, "output: value decoding: %s\n", err) + return + } + mvccValBytes, err := mvccVal.Value.GetBytes() + if err != nil { + fmt.Fprintf(b, "output: value decoding: %s\n", err) + return + } fmt.Fprintf(b, "output: value k=%s ts=%s v=%s\n", - string(k1.Key), k1.Timestamp, string(v1)) + string(k1.Key), k1.Timestamp, string(mvccValBytes)) } // TestIntentInterleavingIter is a datadriven test consisting of two commands: @@ -299,7 +309,8 @@ func TestIntentInterleavingIter(t *testing.T) { var value string d.ScanArgs(t, "v", &value) mvccKey := MVCCKey{Key: key, Timestamp: ts} - if err := batch.PutMVCC(mvccKey, []byte(value)); err != nil { + mvccValue := MVCCValue{Value: roachpb.MakeValueFromString(value)} + if err := batch.PutMVCC(mvccKey, mvccValue); err != nil { return err.Error() } } @@ -567,7 +578,7 @@ func writeRandomData( if kv.Key.Timestamp.IsEmpty() { panic("timestamp should not be empty") } else { - require.NoError(t, batch.PutMVCC(kv.Key, kv.Value)) + require.NoError(t, batch.PutRawMVCC(kv.Key, kv.Value)) } } require.NoError(t, batch.Commit(true)) @@ -785,7 +796,9 @@ func writeBenchData( } for j := versionsPerKey; j >= 1; j-- { require.NoError(b, batch.PutMVCC( - MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(j)}}, []byte("value"))) + MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(j)}}, + MVCCValue{Value: roachpb.MakeValueFromString("value")}, + )) } } require.NoError(b, batch.Commit(true)) diff --git a/pkg/storage/multi_iterator_test.go b/pkg/storage/multi_iterator_test.go index eb71607ad801..aa84e9acb46e 100644 --- a/pkg/storage/multi_iterator_test.go +++ b/pkg/storage/multi_iterator_test.go @@ -108,7 +108,7 @@ func TestMultiIterator(t *testing.T) { t.Fatalf("%+v", err) } } else { - if err := batch.PutMVCC(MVCCKey{Key: k, Timestamp: ts}, v); err != nil { + if err := batch.PutRawMVCC(MVCCKey{Key: k, Timestamp: ts}, v); err != nil { t.Fatalf("%+v", err) } } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index c7c5ac51fbf6..252a14f9805a 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -95,7 +95,9 @@ func emptyKeyError() error { // MVCCKeyValue contains the raw bytes of the value for a key. type MVCCKeyValue struct { - Key MVCCKey + Key MVCCKey + // if Key.IsValue(), Value is an encoded MVCCValue. + // else, Value is an encoded MVCCMetadata. Value []byte } @@ -224,6 +226,7 @@ func updateStatsOnMerge(key roachpb.Key, valSize, nowNanos int64) enginepb.MVCCS // If this value is an intent, updates the intent counters. func updateStatsOnPut( key roachpb.Key, + prevIsValue bool, prevValSize int64, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize int64, orig, meta *enginepb.MVCCMetadata, @@ -317,8 +320,6 @@ func updateStatsOnPut( // Note that when meta.Timestamp equals orig.Timestamp, the computation is // moot, which is something our callers may exploit (since retrieving the // previous version is not for free). - prevIsValue := prevValSize > 0 - if prevIsValue { // If the previous value (exists and) was not a deletion tombstone, make it // live at orig.Timestamp. We don't have to do anything if there is a @@ -385,6 +386,7 @@ func updateStatsOnPut( // counters if commit=true. func updateStatsOnResolve( key roachpb.Key, + prevIsValue bool, prevValSize int64, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize int64, orig, meta *enginepb.MVCCMetadata, @@ -394,7 +396,8 @@ func updateStatsOnResolve( if isSysLocal(key) { // Straightforward: old contribution goes, new contribution comes, and we're done. - ms.SysBytes += (metaKeySize + metaValSize) - (origMetaValSize + origMetaKeySize) + ms.SysBytes -= origMetaKeySize + origMetaValSize + orig.KeyBytes + orig.ValBytes + ms.SysBytes += metaKeySize + metaValSize + meta.KeyBytes + meta.ValBytes return ms } @@ -447,7 +450,6 @@ func updateStatsOnResolve( // and so in that case the caller may simply pass prevValSize=0 and can // skip computing that quantity in the first place. _ = updateStatsOnPut - prevIsValue := prevValSize > 0 if prevIsValue { ms.LiveBytes += MVCCVersionTimestampSize + prevValSize @@ -857,7 +859,7 @@ func mvccGetMetadata( if !iterAlreadyPositioned { iter.SeekGE(metaKey) } - if ok, err := iter.Valid(); !ok { + if ok, err = iter.Valid(); !ok { return false, 0, 0, err } @@ -874,13 +876,22 @@ func mvccGetMetadata( int64(len(iter.UnsafeValue())), nil } + unsafeValRaw := iter.UnsafeValue() + unsafeVal, ok, err := tryDecodeSimpleMVCCValue(unsafeValRaw) + if !ok && err == nil { + unsafeVal, err = decodeExtendedMVCCValue(unsafeValRaw) + } + if err != nil { + return false, 0, 0, err + } + meta.Reset() // For values, the size of keys is always accounted for as // MVCCVersionTimestampSize. The size of the metadata key is // accounted for separately. meta.KeyBytes = MVCCVersionTimestampSize - meta.ValBytes = int64(len(iter.UnsafeValue())) - meta.Deleted = meta.ValBytes == 0 + meta.ValBytes = int64(len(unsafeValRaw)) + meta.Deleted = unsafeVal.IsTombstone() meta.Timestamp = unsafeKey.Timestamp.ToLegacyTimestamp() return true, int64(unsafeKey.EncodedSize()) - meta.KeyBytes, 0, nil } @@ -892,8 +903,8 @@ func mvccGetMetadata( type putBuffer struct { meta enginepb.MVCCMetadata newMeta enginepb.MVCCMetadata - ts hlc.LegacyTimestamp - tmpbuf []byte + ts hlc.LegacyTimestamp // avoids heap allocations + tmpbuf []byte // avoids heap allocations } var putBufferPool = sync.Pool{ @@ -1064,19 +1075,11 @@ func mvccPutUsingIter( localTimestamp hlc.ClockTimestamp, value roachpb.Value, txn *roachpb.Transaction, - valueFn func(optionalValue) ([]byte, error), + valueFn func(optionalValue) (roachpb.Value, error), ) error { - var rawBytes []byte - if valueFn == nil { - if !value.Timestamp.IsEmpty() { - return errors.Errorf("cannot have timestamp set in value on Put") - } - rawBytes = value.RawBytes - } - buf := newPutBuffer() - err := mvccPutInternal(ctx, writer, iter, ms, key, timestamp, localTimestamp, rawBytes, txn, buf, valueFn) + err := mvccPutInternal(ctx, writer, iter, ms, key, timestamp, localTimestamp, value, txn, buf, valueFn) // Using defer would be more convenient, but it is measurably slower. buf.release() @@ -1094,11 +1097,11 @@ func maybeGetValue( ctx context.Context, iter MVCCIterator, key roachpb.Key, - value []byte, + value roachpb.Value, exists bool, readTimestamp hlc.Timestamp, - valueFn func(optionalValue) ([]byte, error), -) ([]byte, error) { + valueFn func(optionalValue) (roachpb.Value, error), +) (roachpb.Value, error) { // If a valueFn is specified, read existing value using the iter. if valueFn == nil { return value, nil @@ -1108,7 +1111,7 @@ func maybeGetValue( var err error exVal, _, err = mvccGet(ctx, iter, key, readTimestamp, MVCCGetOptions{Tombstones: true}) if err != nil { - return nil, err + return roachpb.Value{}, err } } return valueFn(exVal) @@ -1164,9 +1167,9 @@ func replayTransactionalWrite( meta *enginepb.MVCCMetadata, key roachpb.Key, timestamp hlc.Timestamp, - value []byte, + value roachpb.Value, txn *roachpb.Transaction, - valueFn func(optionalValue) ([]byte, error), + valueFn func(optionalValue) (roachpb.Value, error), ) error { var found bool var writtenValue []byte @@ -1236,9 +1239,9 @@ func replayTransactionalWrite( // To ensure the transaction is idempotent, we must assert that the // calculated value on this replay is the same as the one we've previously // written. - if !bytes.Equal(value, writtenValue) { + if !bytes.Equal(value.RawBytes, writtenValue) { return errors.AssertionFailedf("transaction %s with sequence %d has a different value %+v after recomputing from what was written: %+v", - txn.ID, txn.Sequence, value, writtenValue) + txn.ID, txn.Sequence, value.RawBytes, writtenValue) } return nil } @@ -1284,18 +1287,20 @@ func mvccPutInternal( key roachpb.Key, timestamp hlc.Timestamp, localTimestamp hlc.ClockTimestamp, - value []byte, + value roachpb.Value, txn *roachpb.Transaction, buf *putBuffer, - valueFn func(optionalValue) ([]byte, error), + valueFn func(optionalValue) (roachpb.Value, error), ) error { if len(key) == 0 { return emptyKeyError() } - if timestamp.WallTime < 0 { return errors.Errorf("cannot write to %q at timestamp %s", key, timestamp) } + if !value.Timestamp.IsEmpty() { + return errors.Errorf("cannot have timestamp set in value") + } metaKey := MakeMVCCMetadataKey(key) ok, origMetaKeySize, origMetaValSize, err := @@ -1320,10 +1325,10 @@ func mvccPutInternal( if value, err = maybeGetValue(ctx, iter, key, value, ok, timestamp, valueFn); err != nil { return err } - if value == nil { + if !value.IsPresent() { metaKeySize, metaValSize, err = 0, 0, writer.ClearUnversioned(metaKey.Key) } else { - buf.meta = enginepb.MVCCMetadata{RawBytes: value} + buf.meta = enginepb.MVCCMetadata{RawBytes: value.RawBytes} metaKeySize, metaValSize, err = buf.putInlineMeta(writer, metaKey, &buf.meta) } if ms != nil { @@ -1370,6 +1375,7 @@ func mvccPutInternal( } var maybeTooOldErr error + var prevIsValue bool var prevValSize int64 if ok { // There is existing metadata for this key; ensure our write is permitted. @@ -1466,24 +1472,32 @@ func mvccPutInternal( // delete the old intent, taking care with MVCC stats. logicalOp = MVCCUpdateIntentOpType if metaTimestamp.Less(writeTimestamp) { + versionKey := metaKey + versionKey.Timestamp = metaTimestamp + { // If the older write intent has a version underneath it, we need to // read its size because its GCBytesAge contribution may change as we // move the intent above it. A similar phenomenon occurs in // MVCCResolveWriteIntent. - latestKey := MVCCKey{Key: key, Timestamp: metaTimestamp} - _, prevUnsafeVal, haveNextVersion, err := unsafeNextVersion(iter, latestKey) + _, prevUnsafeVal, haveNextVersion, err := unsafeNextVersion(iter, versionKey) if err != nil { return err } if haveNextVersion { + prevVal, ok, err := tryDecodeSimpleMVCCValue(prevUnsafeVal) + if !ok && err == nil { + prevVal, err = decodeExtendedMVCCValue(prevUnsafeVal) + } + if err != nil { + return err + } + prevIsValue = prevVal.Value.IsPresent() prevValSize = int64(len(prevUnsafeVal)) } iter = nil // prevent accidental use below } - versionKey := metaKey - versionKey.Timestamp = metaTimestamp if err := writer.ClearMVCC(versionKey); err != nil { return err } @@ -1575,30 +1589,39 @@ func mvccPutInternal( } } + versionKey := metaKey + versionKey.Timestamp = writeTimestamp + + versionValue := MVCCValue{} + versionValue.Value = value + versionValue.LocalTimestamp = localTimestamp + if !versionValue.LocalTimestampNeeded(versionKey) { + versionValue.LocalTimestamp = hlc.ClockTimestamp{} + } + + // Write the mvcc metadata now that we have sizes for the latest + // versioned value. For values, the size of keys is always accounted + // for as MVCCVersionTimestampSize. The size of the metadata key is + // accounted for separately. + newMeta := &buf.newMeta { var txnMeta *enginepb.TxnMeta if txn != nil { txnMeta = &txn.TxnMeta // If we bumped the WriteTimestamp, we update both the TxnMeta and the // MVCCMetadata.Timestamp. - if txnMeta.WriteTimestamp != writeTimestamp { + if txnMeta.WriteTimestamp != versionKey.Timestamp { txnMetaCpy := *txnMeta - txnMetaCpy.WriteTimestamp.Forward(writeTimestamp) + txnMetaCpy.WriteTimestamp.Forward(versionKey.Timestamp) txnMeta = &txnMetaCpy } } - buf.newMeta.Txn = txnMeta - buf.newMeta.Timestamp = writeTimestamp.ToLegacyTimestamp() + newMeta.Txn = txnMeta } - newMeta := &buf.newMeta - - // Write the mvcc metadata now that we have sizes for the latest - // versioned value. For values, the size of keys is always accounted - // for as MVCCVersionTimestampSize. The size of the metadata key is - // accounted for separately. + newMeta.Timestamp = versionKey.Timestamp.ToLegacyTimestamp() newMeta.KeyBytes = MVCCVersionTimestampSize - newMeta.ValBytes = int64(len(value)) - newMeta.Deleted = value == nil + newMeta.ValBytes = int64(encodedMVCCValueSize(versionValue)) + newMeta.Deleted = versionValue.IsTombstone() var metaKeySize, metaValSize int64 if newMeta.Txn != nil { @@ -1629,15 +1652,13 @@ func mvccPutInternal( // that the meta key is always ordered before the value key and that // RocksDB's skiplist memtable implementation includes a fast-path for // sequential insertion patterns. - versionKey := metaKey - versionKey.Timestamp = writeTimestamp - if err := writer.PutMVCC(versionKey, value); err != nil { + if err := writer.PutMVCC(versionKey, versionValue); err != nil { return err } // Update MVCC stats. if ms != nil { - ms.Add(updateStatsOnPut(key, prevValSize, origMetaKeySize, origMetaValSize, + ms.Add(updateStatsOnPut(key, prevIsValue, prevValSize, origMetaKeySize, origMetaValSize, metaKeySize, metaValSize, meta, newMeta)) } @@ -1680,11 +1701,11 @@ func MVCCIncrement( var int64Val int64 var newInt64Val int64 - valueFn := func(value optionalValue) ([]byte, error) { + valueFn := func(value optionalValue) (roachpb.Value, error) { if value.IsPresent() { var err error if int64Val, err = value.GetInt(); err != nil { - return nil, errors.Errorf("key %q does not contain an integer value", key) + return roachpb.Value{}, errors.Errorf("key %q does not contain an integer value", key) } } @@ -1692,7 +1713,7 @@ func MVCCIncrement( if willOverflow(int64Val, inc) { // Return the old value, since we've failed to modify it. newInt64Val = int64Val - return nil, &roachpb.IntegerOverflowError{ + return roachpb.Value{}, &roachpb.IntegerOverflowError{ Key: key, CurrentValue: int64Val, IncrementValue: inc, @@ -1703,7 +1724,7 @@ func MVCCIncrement( newValue := roachpb.Value{} newValue.SetInt(newInt64Val) newValue.InitChecksum(key) - return newValue.RawBytes, nil + return newValue, nil } err := mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) @@ -1794,19 +1815,19 @@ func mvccConditionalPutUsingIter( allowNoExisting CPutMissingBehavior, txn *roachpb.Transaction, ) error { - valueFn := func(existVal optionalValue) ([]byte, error) { + valueFn := func(existVal optionalValue) (roachpb.Value, error) { if expValPresent, existValPresent := len(expBytes) != 0, existVal.IsPresent(); expValPresent && existValPresent { if !bytes.Equal(expBytes, existVal.TagAndDataBytes()) { - return nil, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &roachpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } } else if expValPresent != existValPresent && (existValPresent || !bool(allowNoExisting)) { - return nil, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &roachpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } - return value.RawBytes, nil + return value, nil } return mvccPutUsingIter(ctx, writer, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) } @@ -1871,20 +1892,20 @@ func mvccInitPutUsingIter( failOnTombstones bool, txn *roachpb.Transaction, ) error { - valueFn := func(existVal optionalValue) ([]byte, error) { + valueFn := func(existVal optionalValue) (roachpb.Value, error) { if failOnTombstones && existVal.IsTombstone() { // We found a tombstone and failOnTombstones is true: fail. - return nil, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &roachpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } if existVal.IsPresent() && !existVal.EqualTagAndData(value) { // The existing value does not match the supplied value. - return nil, &roachpb.ConditionFailedError{ + return roachpb.Value{}, &roachpb.ConditionFailedError{ ActualValue: existVal.ToPointer(), } } - return value.RawBytes, nil + return value, nil } return mvccPutUsingIter(ctx, rw, iter, ms, key, timestamp, localTimestamp, noValue, txn, valueFn) } @@ -2091,8 +2112,7 @@ func MVCCClearTimeRange( defer iter.Close() var clearedMetaKey MVCCKey - var clearedMeta enginepb.MVCCMetadata - var restoredMeta enginepb.MVCCMetadata + var clearedMeta, restoredMeta enginepb.MVCCMetadata iter.SeekGE(MVCCKey{Key: key}) for { if ok, err := iter.Valid(); err != nil { @@ -2102,16 +2122,20 @@ func MVCCClearTimeRange( } k := iter.UnsafeKey() + vRaw := iter.UnsafeValue() + v, err := DecodeMVCCValue(vRaw) + if err != nil { + return nil, err + } if len(clearedMetaKey.Key) > 0 { metaKeySize := int64(clearedMetaKey.EncodedSize()) if bytes.Equal(clearedMetaKey.Key, k.Key) { // Since the key matches, our previous clear "restored" this revision of // the this key, so update the stats with this as the "restored" key. - valueSize := int64(len(iter.Value())) restoredMeta.KeyBytes = MVCCVersionTimestampSize - restoredMeta.Deleted = valueSize == 0 - restoredMeta.ValBytes = valueSize + restoredMeta.ValBytes = int64(len(vRaw)) + restoredMeta.Deleted = v.IsTombstone() restoredMeta.Timestamp = k.Timestamp.ToLegacyTimestamp() ms.Add(updateStatsOnClear( @@ -2136,8 +2160,8 @@ func MVCCClearTimeRange( clearMatchingKey(k) clearedMetaKey.Key = append(clearedMetaKey.Key[:0], k.Key...) clearedMeta.KeyBytes = MVCCVersionTimestampSize - clearedMeta.ValBytes = int64(len(iter.UnsafeValue())) - clearedMeta.Deleted = clearedMeta.ValBytes == 0 + clearedMeta.ValBytes = int64(len(vRaw)) + clearedMeta.Deleted = v.IsTombstone() clearedMeta.Timestamp = k.Timestamp.ToLegacyTimestamp() // Move the iterator to the next key/value in linear iteration even if it @@ -2215,7 +2239,7 @@ func MVCCDeleteRange( var keys []roachpb.Key for i, kv := range res.KVs { if err := mvccPutInternal( - ctx, rw, iter, ms, kv.Key, timestamp, localTimestamp, nil, txn, buf, nil, + ctx, rw, iter, ms, kv.Key, timestamp, localTimestamp, noValue, txn, buf, nil, ); err != nil { return nil, nil, 0, err } @@ -3005,9 +3029,12 @@ func mvccResolveWriteIntent( // can be considered empty and marked for removal (removeIntent = true). // If only part of the intent history was rolled back, but the intent still // remains, the rolledBackVal is set to a non-nil value. - var rolledBackVal []byte + var rolledBackVal *MVCCValue if len(intent.IgnoredSeqNums) > 0 { // NOTE: mvccMaybeRewriteIntentHistory mutates its meta argument. + // TODO(nvanbenschoten): this is an awkward interface. We shouldn't + // be mutating meta and we shouldn't be restoring the previous value + // here. Instead, this should all be handled down below. var removeIntent bool removeIntent, rolledBackVal, err = mvccMaybeRewriteIntentHistory(ctx, rw, intent.IgnoredSeqNums, meta, latestKey) if err != nil { @@ -3051,6 +3078,9 @@ func mvccResolveWriteIntent( // is because removeIntent implies rolledBackVal == nil, pushed == false, and // commit == false. if commit || pushed || rolledBackVal != nil { + buf.newMeta = *meta + newMeta := &buf.newMeta + // The intent might be committing at a higher timestamp, or it might be // getting pushed. newTimestamp := intent.Txn.WriteTimestamp @@ -3063,35 +3093,13 @@ func mvccResolveWriteIntent( metaTimestamp, newTimestamp, commit, pushed, rolledBackVal != nil) } - buf.newMeta = *meta - // Set the timestamp for upcoming write (or at least the stats update). - buf.newMeta.Timestamp = newTimestamp.ToLegacyTimestamp() - buf.newMeta.Txn.WriteTimestamp = newTimestamp - - // Update or remove the metadata key. - var metaKeySize, metaValSize int64 - if !commit { - // Keep existing intent if we're updating it. We update the existing - // metadata's timestamp instead of using the supplied intent meta to avoid - // overwriting a newer epoch (see comments above). The pusher's job isn't - // to do anything to update the intent but to move the timestamp forward, - // even if it can. - metaKeySize, metaValSize, err = buf.putIntentMeta( - ctx, rw, metaKey, &buf.newMeta, true /* alreadyExists */) - } else { - metaKeySize = int64(metaKey.EncodedSize()) - err = rw.ClearIntent(metaKey.Key, canSingleDelHelper.onCommitIntent(), meta.Txn.ID) - } - if err != nil { - return false, err - } - - // If we're moving the intent's timestamp, adjust stats and - // rewrite it. + // If we're moving the intent's timestamp, rewrite it and adjust stats. + var prevIsValue bool var prevValSize int64 if timestampChanged { - oldKey := MVCCKey{Key: intent.Key, Timestamp: metaTimestamp} - newKey := MVCCKey{Key: intent.Key, Timestamp: newTimestamp} + oldKey := latestKey + newKey := oldKey + newKey.Timestamp = newTimestamp // Rewrite the versioned value at the new timestamp. iter.SeekGE(oldKey) @@ -3100,16 +3108,39 @@ func mvccResolveWriteIntent( } else if !valid || !iter.UnsafeKey().Equal(oldKey) { return false, errors.Errorf("existing intent value missing: %s", oldKey) } - value := iter.UnsafeValue() + oldValue, err := DecodeMVCCValue(iter.UnsafeValue()) + if err != nil { + return false, err + } // Special case: If mvccMaybeRewriteIntentHistory rolled back to a value // in the intent history and wrote that at oldKey, iter would not be able // to "see" the value since it was created before that value was written // to the engine. In this case, reuse the value returned by // mvccMaybeRewriteIntentHistory. if rolledBackVal != nil { - value = rolledBackVal + oldValue = *rolledBackVal + } + + // The local timestamp does not change during intent resolution unless the + // resolver provides a clock observation from this node that was captured + // while the transaction was still pending, in which case it can be advanced + // to the observed timestamp. + newValue := oldValue + newValue.LocalTimestamp = oldValue.GetLocalTimestamp(oldKey) + newValue.LocalTimestamp.Forward(intent.ClockWhilePending.Timestamp) + if !newValue.LocalTimestampNeeded(newKey) { + newValue.LocalTimestamp = hlc.ClockTimestamp{} } - if err = rw.PutMVCC(newKey, value); err != nil { + + // Update the MVCC metadata with the timestamp for the upcoming write (or + // at least the stats update). + newMeta.Txn.WriteTimestamp = newTimestamp + newMeta.Timestamp = newTimestamp.ToLegacyTimestamp() + newMeta.KeyBytes = MVCCVersionTimestampSize + newMeta.ValBytes = int64(encodedMVCCValueSize(newValue)) + newMeta.Deleted = newValue.IsTombstone() + + if err = rw.PutMVCC(newKey, newValue); err != nil { return false, err } if err = rw.ClearMVCC(oldKey); err != nil { @@ -3128,14 +3159,41 @@ func mvccResolveWriteIntent( if valid, err := iter.Valid(); err != nil { return false, err } else if valid && iter.UnsafeKey().Key.Equal(oldKey.Key) { + unsafeValRaw := iter.UnsafeValue() + prevVal, prevValOK, err := tryDecodeSimpleMVCCValue(unsafeValRaw) + if !prevValOK && err == nil { + prevVal, err = decodeExtendedMVCCValue(unsafeValRaw) + } + if err != nil { + return false, err + } + prevIsValue = prevVal.Value.IsPresent() prevValSize = int64(len(iter.UnsafeValue())) } } + // Update or remove the metadata key. + var metaKeySize, metaValSize int64 + if !commit { + // Keep existing intent if we're updating it. We update the existing + // metadata's timestamp instead of using the supplied intent meta to avoid + // overwriting a newer epoch (see comments above). The pusher's job isn't + // to do anything to update the intent but to move the timestamp forward, + // even if it can. + metaKeySize, metaValSize, err = buf.putIntentMeta( + ctx, rw, metaKey, newMeta, true /* alreadyExists */) + } else { + metaKeySize = int64(metaKey.EncodedSize()) + err = rw.ClearIntent(metaKey.Key, canSingleDelHelper.onCommitIntent(), meta.Txn.ID) + } + if err != nil { + return false, err + } + // Update stat counters related to resolving the intent. if ms != nil { - ms.Add(updateStatsOnResolve(intent.Key, prevValSize, origMetaKeySize, origMetaValSize, - metaKeySize, metaValSize, meta, &buf.newMeta, commit)) + ms.Add(updateStatsOnResolve(intent.Key, prevIsValue, prevValSize, origMetaKeySize, origMetaValSize, + metaKeySize, metaValSize, meta, newMeta, commit)) } // Log the logical MVCC operation. @@ -3178,7 +3236,7 @@ func mvccResolveWriteIntent( nextKey := latestKey.Next() ok = false var unsafeNextKey MVCCKey - var unsafeNextValue []byte + var unsafeNextValueRaw []byte if nextKey.IsValue() { // The latestKey was not the smallest possible timestamp {WallTime: 0, // Logical: 1}. Practically, this is the only case that will occur in @@ -3195,7 +3253,7 @@ func mvccResolveWriteIntent( // particular timestamp. return false, errors.Errorf("expected an MVCC value key: %s", unsafeNextKey) } - unsafeNextValue = iter.UnsafeValue() + unsafeNextValueRaw = iter.UnsafeValue() } else { ok = false } @@ -3217,12 +3275,15 @@ func mvccResolveWriteIntent( } // Get the bytes for the next version so we have size for stat counts. - valueSize := int64(len(unsafeNextValue)) + unsafeNextValue, err := DecodeMVCCValue(unsafeNextValueRaw) + if err != nil { + return false, err + } // Update the keyMetadata with the next version. buf.newMeta = enginepb.MVCCMetadata{ - Deleted: valueSize == 0, + Deleted: unsafeNextValue.IsTombstone(), KeyBytes: MVCCVersionTimestampSize, - ValBytes: valueSize, + ValBytes: int64(len(unsafeNextValueRaw)), } if err = rw.ClearIntent(metaKey.Key, canSingleDelHelper.onAbortIntent(), meta.Txn.ID); err != nil { return false, err @@ -3253,7 +3314,7 @@ func mvccMaybeRewriteIntentHistory( ignoredSeqNums []enginepb.IgnoredSeqNumRange, meta *enginepb.MVCCMetadata, latestKey MVCCKey, -) (remove bool, updatedVal []byte, err error) { +) (remove bool, updatedVal *MVCCValue, err error) { if !enginepb.TxnSeqIsIgnored(meta.Txn.Sequence, ignoredSeqNums) { // The latest write was not ignored. Nothing to do here. We'll // proceed with the intent as usual. @@ -3277,15 +3338,19 @@ func mvccMaybeRewriteIntentHistory( // Otherwise, we place back the write at that history entry // back into the intent. - restoredVal := meta.IntentHistory[i].Value + restoredValRaw := meta.IntentHistory[i].Value + restoredVal, err := DecodeMVCCValue(restoredValRaw) + if err != nil { + return false, nil, err + } meta.Txn.Sequence = meta.IntentHistory[i].Sequence meta.IntentHistory = meta.IntentHistory[:i] - meta.Deleted = len(restoredVal) == 0 - meta.ValBytes = int64(len(restoredVal)) + meta.Deleted = restoredVal.IsTombstone() + meta.ValBytes = int64(len(restoredValRaw)) // And also overwrite whatever was there in storage. err = engine.PutMVCC(latestKey, restoredVal) - return false, restoredVal, err + return false, &restoredVal, err } // IterAndBuf used to pass iterators and buffers between MVCC* calls, allowing @@ -3658,19 +3723,26 @@ func MVCCGarbageCollect( break } if ms != nil { - // FIXME: use prevNanos instead of unsafeIterKey.Timestamp, except - // when it's a deletion. - valSize := int64(len(iter.UnsafeValue())) + unsafeValRaw := iter.UnsafeValue() + unsafeVal, unsafeValOK, err := tryDecodeSimpleMVCCValue(unsafeValRaw) + if !unsafeValOK && err == nil { + unsafeVal, err = decodeExtendedMVCCValue(unsafeValRaw) + } + if err != nil { + return err + } + + keySize := MVCCVersionTimestampSize + valSize := int64(len(unsafeValRaw)) // A non-deletion becomes non-live when its newer neighbor shows up. // A deletion tombstone becomes non-live right when it is created. fromNS := prevNanos - if valSize == 0 { + if unsafeVal.IsTombstone() { fromNS = unsafeIterKey.Timestamp.WallTime } - ms.Add(updateStatsOnGC(gcKey.Key, MVCCVersionTimestampSize, - valSize, nil, fromNS)) + ms.Add(updateStatsOnGC(gcKey.Key, keySize, valSize, nil, fromNS)) } count++ if err := rw.ClearMVCC(unsafeIterKey); err != nil { @@ -3864,10 +3936,22 @@ func ComputeStatsForRange( if implicitMeta { // No MVCCMetadata entry for this series of keys. + var isTombstone bool + { + mvccValue, ok, err := tryDecodeSimpleMVCCValue(unsafeValue) + if !ok && err == nil { + mvccValue, err = decodeExtendedMVCCValue(unsafeValue) + } + if err != nil { + return ms, errors.Wrap(err, "unable to decode MVCCValue") + } + isTombstone = mvccValue.IsTombstone() + } + meta.Reset() meta.KeyBytes = MVCCVersionTimestampSize meta.ValBytes = int64(len(unsafeValue)) - meta.Deleted = len(unsafeValue) == 0 + meta.Deleted = isTombstone meta.Timestamp.WallTime = unsafeKey.Timestamp.WallTime } @@ -3941,7 +4025,18 @@ func ComputeStatsForRange( accrueGCAgeNanos = meta.Timestamp.WallTime } else { // Overwritten value. Is it a deletion tombstone? - isTombstone := len(unsafeValue) == 0 + var isTombstone bool + { + mvccValue, ok, err := tryDecodeSimpleMVCCValue(unsafeValue) + if !ok && err == nil { + mvccValue, err = decodeExtendedMVCCValue(unsafeValue) + } + if err != nil { + return ms, errors.Wrap(err, "unable to decode MVCCValue") + } + isTombstone = mvccValue.IsTombstone() + } + if isTombstone { // The contribution of the tombstone picks up GCByteAge from its own timestamp on. ms.GCBytesAge += totalBytes * (nowNanos/1e9 - unsafeKey.Timestamp.WallTime/1e9) diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 702040da32b2..5b5e71ae7fa9 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -52,7 +52,7 @@ import ( // txn_advance t= ts=[,] // txn_status t= status= // -// resolve_intent t= k= [status=] +// resolve_intent t= k= [status=] [clockWhilePending=[,]] // check_intent k= [none] // // cput [t=] [ts=[,]] [localTs=[,]] [resolve [status=]] k= v= [raw] [cond=] @@ -124,7 +124,12 @@ func TestMVCCHistories(t *testing.T) { buf.Printf("meta: %v -> %+v\n", r.Key, &meta) } } else { - buf.Printf("data: %v -> %s\n", r.Key, roachpb.Value{RawBytes: r.Value}.PrettyPrint()) + val, err := DecodeMVCCValue(r.Value) + if err != nil { + buf.Printf("data: %v -> error decoding value %v: %v\n", r.Key, r.Value, err) + } else { + buf.Printf("data: %v -> %s\n", r.Key, val) + } } return nil }) @@ -543,14 +548,20 @@ func cmdResolveIntent(e *evalCtx) error { txn := e.getTxn(mandatory) key := e.getKey() status := e.getTxnStatus() - return e.resolveIntent(e.tryWrapForIntentPrinting(e.engine), key, txn, status) + clockWhilePending := hlc.ClockTimestamp(e.getTsWithName("clockWhilePending")) + return e.resolveIntent(e.tryWrapForIntentPrinting(e.engine), key, txn, status, clockWhilePending) } func (e *evalCtx) resolveIntent( - rw ReadWriter, key roachpb.Key, txn *roachpb.Transaction, resolveStatus roachpb.TransactionStatus, + rw ReadWriter, + key roachpb.Key, + txn *roachpb.Transaction, + resolveStatus roachpb.TransactionStatus, + clockWhilePending hlc.ClockTimestamp, ) error { intent := roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}) intent.Status = resolveStatus + intent.ClockWhilePending = roachpb.ObservedTimestamp{Timestamp: clockWhilePending} _, err := MVCCResolveWriteIntent(e.ctx, rw, nil, intent) return err } @@ -608,7 +619,7 @@ func cmdCPut(e *evalCtx) error { return err } if resolve { - return e.resolveIntent(rw, key, txn, resolveStatus) + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } return nil }) @@ -625,7 +636,7 @@ func cmdDelete(e *evalCtx) error { return err } if resolve { - return e.resolveIntent(rw, key, txn, resolveStatus) + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } return nil }) @@ -658,7 +669,7 @@ func cmdDeleteRange(e *evalCtx) error { } if resolve { - return e.resolveIntent(rw, key, txn, resolveStatus) + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } return nil }) @@ -726,7 +737,7 @@ func cmdIncrement(e *evalCtx) error { } e.results.buf.Printf("inc: current value = %d\n", curVal) if resolve { - return e.resolveIntent(rw, key, txn, resolveStatus) + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } return nil }) @@ -734,14 +745,7 @@ func cmdIncrement(e *evalCtx) error { func cmdMerge(e *evalCtx) error { key := e.getKey() - var value string - e.scanArg("v", &value) - var val roachpb.Value - if e.hasArg("raw") { - val.RawBytes = []byte(value) - } else { - val.SetString(value) - } + val := e.getVal() ts := e.getTs(nil) return e.withWriter("merge", func(rw ReadWriter) error { return MVCCMerge(e.ctx, rw, nil, key, ts, val) @@ -763,7 +767,7 @@ func cmdPut(e *evalCtx) error { return err } if resolve { - return e.resolveIntent(rw, key, txn, resolveStatus) + return e.resolveIntent(rw, key, txn, resolveStatus, hlc.ClockTimestamp{}) } return nil }) diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index 28a2bf88aec6..79c1363b89f9 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -39,13 +39,11 @@ import ( const all, latest = true, false -func makeKVT(key roachpb.Key, value []byte, ts hlc.Timestamp) MVCCKeyValue { - return MVCCKeyValue{Key: MVCCKey{Key: key, Timestamp: ts}, Value: value} +func makeKVT(key roachpb.Key, value roachpb.Value, ts hlc.Timestamp) MVCCKeyValue { + return MVCCKeyValue{Key: MVCCKey{Key: key, Timestamp: ts}, Value: value.RawBytes} } -func makeKVTxn( - key roachpb.Key, val []byte, ts hlc.Timestamp, -) (roachpb.Transaction, roachpb.Value, roachpb.Intent) { +func makeKVTxn(key roachpb.Key, ts hlc.Timestamp) (roachpb.Transaction, roachpb.Intent) { txnID := uuid.MakeV4() txnMeta := enginepb.TxnMeta{ Key: key, @@ -53,12 +51,12 @@ func makeKVTxn( Epoch: 1, WriteTimestamp: ts, } - return roachpb.Transaction{ - TxnMeta: txnMeta, - ReadTimestamp: ts, - }, roachpb.Value{ - RawBytes: val, - }, roachpb.MakeIntent(&txnMeta, key) + txn := roachpb.Transaction{ + TxnMeta: txnMeta, + ReadTimestamp: ts, + } + intent := roachpb.MakeIntent(&txnMeta, key) + return txn, intent } func intents(intents ...roachpb.Intent) []roachpb.Intent { @@ -506,10 +504,10 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { testKey1 = roachpb.Key("/db1") testKey2 = roachpb.Key("/db2") - testValue1 = []byte("val1") - testValue2 = []byte("val2") - testValue3 = []byte("val3") - testValue4 = []byte("val4") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") + testValue3 = roachpb.MakeValueFromString("val3") + testValue4 = roachpb.MakeValueFromString("val4") // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. @@ -525,7 +523,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { kv1_2_2 := makeKVT(testKey1, testValue2, ts2) kv2_2_2 := makeKVT(testKey2, testValue3, ts2) kv2_4_4 := makeKVT(testKey2, testValue4, ts4) - kv1_3Deleted := makeKVT(testKey1, nil, ts3) + kv1_3Deleted := makeKVT(testKey1, roachpb.Value{}, ts3) for _, engineImpl := range mvccEngineImpls { t.Run(engineImpl.name, func(t *testing.T) { @@ -596,8 +594,7 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { }, ReadTimestamp: ts4, } - txn1Val := roachpb.Value{RawBytes: testValue4} - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { t.Fatal(err) } @@ -644,10 +641,10 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { testKey1 = roachpb.Key("/db1") testKey2 = roachpb.Key("/db2") - testValue1 = []byte("val1") - testValue2 = []byte("val2") - testValue3 = []byte("val3") - testValue4 = []byte("val4") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") + testValue3 = roachpb.MakeValueFromString("val3") + testValue4 = roachpb.MakeValueFromString("val4") // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. @@ -662,7 +659,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { kv1_1_1 := makeKVT(testKey1, testValue1, ts1) kv1_2_2 := makeKVT(testKey1, testValue2, ts2) kv2_2_2 := makeKVT(testKey2, testValue3, ts2) - kv1_3Deleted := makeKVT(testKey1, nil, ts3) + kv1_3Deleted := makeKVT(testKey1, roachpb.Value{}, ts3) for _, engineImpl := range mvccEngineImpls { t.Run(engineImpl.name, func(t *testing.T) { @@ -730,8 +727,7 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { }, ReadTimestamp: ts4, } - txn1Val := roachpb.Value{RawBytes: testValue4} - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { t.Fatal(err) } @@ -776,8 +772,8 @@ func TestMVCCIncrementalIteratorInlinePolicy(t *testing.T) { testKey2 = roachpb.Key("/db2") testKey3 = roachpb.Key("/db3") - testValue1 = []byte("val1") - testValue2 = []byte("val2") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. @@ -869,9 +865,9 @@ func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { testKey1 = roachpb.Key("/db1") testKey2 = roachpb.Key("/db2") - testValue1 = []byte("val1") - testValue2 = []byte("val2") - testValue3 = []byte("val3") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") + testValue3 = roachpb.MakeValueFromString("val3") // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. @@ -882,28 +878,12 @@ func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { tsMax = hlc.Timestamp{WallTime: math.MaxInt64, Logical: 0} ) - makeTxn := func(key roachpb.Key, val []byte, ts hlc.Timestamp) (roachpb.Transaction, roachpb.Value, roachpb.Intent) { - txnID := uuid.MakeV4() - txnMeta := enginepb.TxnMeta{ - Key: key, - ID: txnID, - Epoch: 1, - WriteTimestamp: ts, - } - return roachpb.Transaction{ - TxnMeta: txnMeta, - ReadTimestamp: ts, - }, roachpb.Value{ - RawBytes: val, - }, roachpb.MakeIntent(&txnMeta, key) - } - kv1_1_1 := makeKVT(testKey1, testValue1, ts1) kv1_2_2 := makeKVT(testKey1, testValue2, ts2) kv1_3_3 := makeKVT(testKey1, testValue3, ts3) kv2_1_1 := makeKVT(testKey2, testValue1, ts1) kv2_2_2 := makeKVT(testKey2, testValue2, ts2) - txn, val, intent2_2_2 := makeTxn(testKey2, testValue2, ts2) + txn, intent2_2_2 := makeKVTxn(testKey2, ts2) intentErr := &roachpb.WriteIntentError{Intents: []roachpb.Intent{intent2_2_2}} @@ -916,7 +896,7 @@ func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { t.Fatal(err) } } - if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val, &txn); err != nil { + if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, testValue2, &txn); err != nil { t.Fatal(err) } t.Run(engineImpl.name, func(t *testing.T) { @@ -1065,10 +1045,10 @@ func TestMVCCIncrementalIterator(t *testing.T) { testKey1 = roachpb.Key("/db1") testKey2 = roachpb.Key("/db2") - testValue1 = []byte("val1") - testValue2 = []byte("val2") - testValue3 = []byte("val3") - testValue4 = []byte("val4") + testValue1 = roachpb.MakeValueFromString("val1") + testValue2 = roachpb.MakeValueFromString("val2") + testValue3 = roachpb.MakeValueFromString("val3") + testValue4 = roachpb.MakeValueFromString("val4") // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. @@ -1085,7 +1065,7 @@ func TestMVCCIncrementalIterator(t *testing.T) { kv1_4_4 := makeKVT(testKey1, testValue4, ts4) kv1_2_2 := makeKVT(testKey1, testValue2, ts2) kv2_2_2 := makeKVT(testKey2, testValue3, ts2) - kv1Deleted3 := makeKVT(testKey1, nil, ts3) + kv1Deleted3 := makeKVT(testKey1, roachpb.Value{}, ts3) for _, engineImpl := range mvccEngineImpls { t.Run(engineImpl.name+"-latest", func(t *testing.T) { @@ -1120,12 +1100,12 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, latest, kvs(kv1Deleted3, kv2_2_2))) // Exercise intent handling. - txn1, txn1Val, intentErr1 := makeKVTxn(testKey1, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { + txn1, intentErr1 := makeKVTxn(testKey1, ts4) + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { t.Fatal(err) } - txn2, txn2Val, intentErr2 := makeKVTxn(testKey2, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, txn2Val, &txn2); err != nil { + txn2, intentErr2 := makeKVTxn(testKey2, ts4) + if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn2); err != nil { t.Fatal(err) } t.Run("intents-1", @@ -1188,12 +1168,12 @@ func TestMVCCIncrementalIterator(t *testing.T) { t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, all, kvs(kv1Deleted3, kv1_2_2, kv2_2_2))) // Exercise intent handling. - txn1, txn1Val, intentErr1 := makeKVTxn(testKey1, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, txn1Val, &txn1); err != nil { + txn1, intentErr1 := makeKVTxn(testKey1, ts4) + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { t.Fatal(err) } - txn2, txn2Val, intentErr2 := makeKVTxn(testKey2, testValue4, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, txn2Val, &txn2); err != nil { + txn2, intentErr2 := makeKVTxn(testKey2, ts4) + if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn2); err != nil { t.Fatal(err) } // Single intent tests are verifying behavior when intent collection is not enabled. diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 40d75745af12..584fa9791632 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -36,7 +36,14 @@ const ( ) // MVCCKey is a versioned key, distinguished from roachpb.Key with the addition -// of a timestamp. +// of a "version" timestamp. +// +// The version timestamp dictates the key's visibility to readers. Readers with +// read timestamps equal to or greater than the version timestamp observe the +// key. Readers with read timestamps below the version timestamp ignore the key. +// Keys are stored in decreasing version order, with the exception of version +// zero (timestamp 0), which is referred to as a "meta" version and is stored +// before all other versions of the same key. type MVCCKey struct { Key roachpb.Key Timestamp hlc.Timestamp @@ -246,6 +253,7 @@ func encodedMVCCKeyLength(key MVCCKey) int { if key.Timestamp.Logical != 0 || key.Timestamp.Synthetic { keyLen += mvccEncodedTimeLogicalLen if key.Timestamp.Synthetic { + // TODO(nvanbenschoten): stop writing Synthetic timestamps in v23.1. keyLen += mvccEncodedTimeSyntheticLen } } @@ -307,6 +315,7 @@ func decodeMVCCTimestamp(encodedTS []byte) (hlc.Timestamp, error) { case 13: ts.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) ts.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + // TODO(nvanbenschoten): stop writing Synthetic timestamps in v23.1. ts.Synthetic = encodedTS[12] != 0 default: return hlc.Timestamp{}, errors.Errorf("bad timestamp %x", encodedTS) diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 855d5e9fe01d..2d45b747c525 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -35,6 +35,7 @@ import ( // implementations and errors on mismatch with any of them. It is used for global // keys. func assertEq(t *testing.T, rw ReadWriter, debug string, ms, expMS *enginepb.MVCCStats) { + t.Helper() assertEqImpl(t, rw, debug, true /* globalKeys */, ms, expMS) } @@ -137,6 +138,13 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { t.Fatal(err) } + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize += vValHeaderSize // 23 + expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 4e9, LiveBytes: 0, @@ -146,7 +154,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { // The implicit meta record (deletion tombstone) counts for len("a")+1=2. // Two versioned keys count for 2*vKeySize. KeyBytes: mKeySize + 2*vKeySize, - ValBytes: vValSize, // the initial write (10) + ValBytes: vValSize, // No GCBytesAge has been accrued yet, as the value just got non-live at 4s. GCBytesAge: 0, } @@ -185,7 +193,7 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { } mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - mValSize := int64((&enginepb.MVCCMetadata{ // 44 + mValSize := int64((&enginepb.MVCCMetadata{ // 46 Timestamp: ts1.ToLegacyTimestamp(), Deleted: false, Txn: &txn.TxnMeta, @@ -196,11 +204,11 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(44[+2])+12+10 = 68[+2] + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+10 = 68[+2] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 2+12 =14 KeyCount: 1, - ValBytes: mValSize + vValSize, // (44[+2])+10 = 54[+2] + ValBytes: mValSize + vValSize, // (46[+2])+10 = 54[+2] ValCount: 1, IntentCount: 1, SeparatedIntentCount: 1, @@ -220,9 +228,16 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { t.Fatal(err) } + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize += vValHeaderSize // 23 + expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 4e9, - LiveBytes: mKeySize + vKeySize + vValSize, // 2+12+20 = 24 + LiveBytes: mKeySize + vKeySize + vValSize, // 2+12+23 = 37 LiveCount: 1, KeyCount: 1, ValCount: 1, @@ -267,7 +282,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { } mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - mValSize := int64((&enginepb.MVCCMetadata{ // 44 + mValSize := int64((&enginepb.MVCCMetadata{ // 46 Timestamp: ts1.ToLegacyTimestamp(), Deleted: false, Txn: &txn.TxnMeta, @@ -278,11 +293,11 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { expMS := enginepb.MVCCStats{ LastUpdateNanos: 1e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(44[+2])+12+10 = 68[+2] + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+10 = 70[+2] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 2+12 = 14 KeyCount: 1, - ValBytes: mValSize + vValSize, // (44[+2])+10 = 54[+2] + ValBytes: mValSize + vValSize, // (46[+2])+10 = 54[+2] ValCount: 1, IntentAge: 0, IntentCount: 1, @@ -300,12 +315,19 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { ); err != nil { t.Fatal(err) } - // Account for removal of TxnDidNotUpdateMeta + // Account for removal of TxnDidNotUpdateMeta. mValSize -= 2 + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize += vValHeaderSize // 23 + expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 4e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+44+12+20 = 78 + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+54+12+23 = 91 LiveCount: 1, KeyCount: 1, ValCount: 1, @@ -313,7 +335,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { // One versioned key counts for vKeySize. KeyBytes: mKeySize + vKeySize, // The intent is still there, so we see mValSize. - ValBytes: vValSize + mValSize, // 44+10 = 54 + ValBytes: mValSize + vValSize, // 54+23 = 69 IntentAge: 0, // this was once erroneously positive IntentCount: 1, // still there SeparatedIntentCount: 1, @@ -647,11 +669,18 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { t.Fatal(err) } + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts2)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize := vValHeaderSize + 0 // tombstone, so just a header + expAggMS := enginepb.MVCCStats{ LastUpdateNanos: 3e9, KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 KeyCount: 1, - ValBytes: 0, + ValBytes: vValSize, ValCount: 2, IntentCount: 0, IntentBytes: 0, @@ -980,7 +1009,7 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { } mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - m1ValSize := int64((&enginepb.MVCCMetadata{ // 44 + m1ValSize := int64((&enginepb.MVCCMetadata{ // 46 Timestamp: ts201.ToLegacyTimestamp(), Txn: &txn.TxnMeta, }).Size()) @@ -990,11 +1019,11 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { expMS := enginepb.MVCCStats{ LastUpdateNanos: 2e9 + 1, - LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(44[+2])+12+10 = 68[+2] + LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(46[+2])+12+10 = 68[+2] LiveCount: 1, KeyBytes: mKeySize + vKeySize, // 14 KeyCount: 1, - ValBytes: m1ValSize + vValSize, // (44[+2])+10 = 54[+2] + ValBytes: m1ValSize + vValSize, // (46[+2])+10 = 54[+2] ValCount: 1, IntentCount: 1, SeparatedIntentCount: 1, diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 6060927203a2..8598b3c9c656 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -960,23 +960,24 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := engine.PutMVCC(MVCCKey{testKey1, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + value := MVCCValue{Value: roachpb.MakeValueFromString("foobar")} + err := engine.PutMVCC(MVCCKey{Key: testKey1, Timestamp: hlc.Timestamp{WallTime: 5}}, value) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey2, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{Key: testKey2, Timestamp: hlc.Timestamp{WallTime: 5}}, value) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey2, hlc.Timestamp{WallTime: 3}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{Key: testKey2, Timestamp: hlc.Timestamp{WallTime: 3}}, value) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey3, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{Key: testKey3, Timestamp: hlc.Timestamp{WallTime: 5}}, value) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey4, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{Key: testKey4, Timestamp: hlc.Timestamp{WallTime: 5}}, value) if err != nil { t.Fatal(err) } @@ -997,7 +998,7 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { UpperBound: testKey5, }) defer iter.Close() - iter.SeekGE(MVCCKey{testKey1, hlc.Timestamp{WallTime: 5}}) + iter.SeekGE(MVCCKey{Key: testKey1, Timestamp: hlc.Timestamp{WallTime: 5}}) iter.Next() // key2/5 // Lay down an intent on key3, which will go at key3/0 and sort before key3/5. @@ -4137,8 +4138,8 @@ func TestRandomizedMVCCResolveWriteIntentRange(t *testing.T) { // non-decreasing due to tsIndex. versions := rng.Intn(3) + 1 for j := 0; j < versions; j++ { - val := generateBytes(rng, 20, 30) - put.values = append(put.values, roachpb.Value{RawBytes: val}) + val := roachpb.MakeValueFromBytes(generateBytes(rng, 20, 30)) + put.values = append(put.values, val) put.seqs = append(put.seqs, seq) seq++ index := rng.Intn(len(timestamps)) @@ -4261,8 +4262,8 @@ func TestRandomizedSavepointRollbackAndIntentResolution(t *testing.T) { key: key, } for j := 0; j < 2; j++ { - val := generateBytes(rng, 20, 30) - put.values = append(put.values, roachpb.Value{RawBytes: val}) + val := roachpb.MakeValueFromBytes(generateBytes(rng, 20, 30)) + put.values = append(put.values, val) put.seqs = append(put.seqs, seq) seq++ put.writeTS = append(put.writeTS, timestamps[j]) @@ -4310,8 +4311,8 @@ func TestRandomizedSavepointRollbackAndIntentResolution(t *testing.T) { } // Do another put for all these keys. These will also be in the memtable. for i := 0; i < 100; i++ { - puts[i].values = append(puts[i].values[:0], - roachpb.Value{RawBytes: generateBytes(rng, 2, 3)}) + val := roachpb.MakeValueFromBytes(generateBytes(rng, 2, 3)) + puts[i].values = append(puts[i].values[:0], val) puts[i].seqs = append(puts[i].seqs[:0], seq) seq++ puts[i].writeTS = append(puts[i].writeTS[:0], timestamps[2]) diff --git a/pkg/storage/mvcc_value.go b/pkg/storage/mvcc_value.go new file mode 100644 index 000000000000..86ea768eb0b6 --- /dev/null +++ b/pkg/storage/mvcc_value.go @@ -0,0 +1,245 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +import ( + "encoding/binary" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" +) + +const ( + extendedLenSize = 4 // also checksumSize for roachpb.Value + tagPos = extendedLenSize + tagSize = 1 + extendedPreludeSize = extendedLenSize + tagSize + + extendedEncodingSentinel = byte(roachpb.ValueType_MVCC_EXTENDED_ENCODING_SENTINEL) +) + +// MVCCValue is a versioned value, stored at an associated MVCCKey with a +// non-zero version timestamp. +// +// MVCCValue wraps a roachpb.Value and extends it with MVCC-level metadata which +// is stored in an enginepb.MVCCValueHeader struct. +// +// The mvcc value has a "simple" and an "extended" encoding scheme, depending on +// whether the value's header is empty or not. If the value's header is empty, +// it is omitted in the encoding and the mvcc value's encoding is identical to +// that of roachpb.Value. This provided backwards compatibility and ensures that +// the MVCCValue optimizes away in the common case. If the value's header is not +// empty, it is prepended to the roachpb.Value encoding. The encoding scheme's +// variants are: +// +// Simple (identical to the roachpb.Value encoding): +// +// <4-byte-checksum><1-byte-tag> +// +// Extended (header prepended to roachpb.Value encoding): +// +// <4-byte-header-len><1-byte-sentinel><4-byte-checksum><1-byte-tag> +// +// The two encoding scheme variants are distinguished using the 5th byte, which +// is either the roachpb.Value tag (which has many values) or a sentinel tag not +// used by the roachpb.Value encoding which indicates the extended encoding +// scheme. +// +// For a deletion tombstone, the encoding of roachpb.Value is special cased to +// be empty, i.e., no checksum, tag, or encoded-data. In that case the extended +// encoding above is simply: +// +// <4-byte-header-len><1-byte-sentinel> +// +// To identify a deletion tombstone from an encoded MVCCValue, callers should +// decode the value using DecodeMVCCValue and then use the IsTombstone method. +// For example: +// +// valRaw := iter.UnsafeValue() +// val, err := DecodeMVCCValue(valRaw) +// if err != nil { ... } +// isTombstone := val.IsTombstone() +// +type MVCCValue struct { + enginepb.MVCCValueHeader + Value roachpb.Value +} + +// IsTombstone returns whether the MVCCValue represents a deletion tombstone. +func (v MVCCValue) IsTombstone() bool { + return len(v.Value.RawBytes) == 0 +} + +// LocalTimestampNeeded returns whether the MVCCValue's local timestamp is +// needed, or whether it can be implied by (i.e. set to the same value as) +// its key's version timestamp. +func (v MVCCValue) LocalTimestampNeeded(k MVCCKey) bool { + // If the local timestamp is empty, it is assumed to be equal to the key's + // version timestamp and so the local timestamp is not needed. + return !v.LocalTimestamp.IsEmpty() && + // If the local timestamp is not empty, it is safe for the local clock + // timestamp to be rounded down, as this will simply lead to additional + // uncertainty restarts. In such cases, the local timestamp is not needed. + // However, it is not safe for the local clock timestamp to be rounded up, + // as this could lead to stale reads. As a result, in such cases, the local + // timestamp is needed and cannot be implied by the version timestamp. + v.LocalTimestamp.ToTimestamp().Less(k.Timestamp) +} + +// GetLocalTimestamp returns the MVCCValue's local timestamp. If the local +// timestamp is not set explicitly, its implicit value is taken from the +// provided MVCCKey and returned. +func (v MVCCValue) GetLocalTimestamp(k MVCCKey) hlc.ClockTimestamp { + if v.LocalTimestamp.IsEmpty() { + if k.Timestamp.Synthetic { + // A synthetic version timestamp means that the version timestamp is + // disconnected from real time and did not come from an HLC clock on the + // leaseholder that wrote the value or from somewhere else in the system. + // As a result, the version timestamp cannot be cast to a clock timestamp, + // so we return min_clock_timestamp instead. The effect of this is that + // observed timestamps can not be used to avoid uncertainty retries for + // values without a local timestamp and with a synthetic version + // timestamp. + return hlc.MinClockTimestamp + } + return hlc.ClockTimestamp(k.Timestamp) + } + return v.LocalTimestamp +} + +// String implements the fmt.Stringer interface. +func (v MVCCValue) String() string { + return redact.StringWithoutMarkers(v) +} + +// SafeFormat implements the redact.SafeFormatter interface. +func (v MVCCValue) SafeFormat(w redact.SafePrinter, _ rune) { + if v.MVCCValueHeader != (enginepb.MVCCValueHeader{}) { + w.Printf("vheader{") + if !v.LocalTimestamp.IsEmpty() { + w.Printf(" localTs=%s", v.LocalTimestamp) + } + w.Printf(" } ") + } + w.Print(v.Value.PrettyPrint()) +} + +// encodedMVCCValueSize returns the size of the MVCCValue when encoded. +//gcassert:inline +func encodedMVCCValueSize(v MVCCValue) int { + if v.MVCCValueHeader == (enginepb.MVCCValueHeader{}) { + return len(v.Value.RawBytes) + } + return extendedPreludeSize + v.MVCCValueHeader.Size() + len(v.Value.RawBytes) +} + +// EncodeMVCCValue encodes an MVCCValue into its Pebble representation. See the +// comment on MVCCValue for a description of the encoding scheme. +//gcassert:inline +func EncodeMVCCValue(v MVCCValue) ([]byte, error) { + if v.MVCCValueHeader == (enginepb.MVCCValueHeader{}) { + // Simple encoding. Use the roachpb.Value encoding directly with no + // modification. No need to re-allocate or copy. + return v.Value.RawBytes, nil + } + // Extended encoding. Wrap the roachpb.Value encoding with a header containing + // MVCC-level metadata. Requires a re-allocation and copy. + return encodeExtendedMVCCValue(v) +} + +// encodeExtendedMVCCValue implements the extended MVCCValue encoding. It is +// split from EncodeMVCCValue to allow that function to qualify for mid-stack +// inlining, which avoids a function call for the simple encoding. +func encodeExtendedMVCCValue(v MVCCValue) ([]byte, error) { + headerLen := v.MVCCValueHeader.Size() + headerSize := extendedPreludeSize + headerLen + valueSize := headerSize + len(v.Value.RawBytes) + + buf := make([]byte, valueSize) + // 4-byte-header-len + binary.BigEndian.PutUint32(buf, uint32(headerLen)) + // 1-byte-sentinel + buf[tagPos] = extendedEncodingSentinel + // mvcc-header + // + // NOTE: we don't use protoutil to avoid passing v.MVCCValueHeader through + // an interface, which would cause a heap allocation and incur the cost of + // dynamic dispatch. + if _, err := v.MVCCValueHeader.MarshalToSizedBuffer(buf[extendedPreludeSize:headerSize]); err != nil { + return nil, errors.Wrap(err, "marshaling MVCCValueHeader") + } + // <4-byte-checksum><1-byte-tag> or empty for tombstone + copy(buf[headerSize:], v.Value.RawBytes) + return buf, nil +} + +// DecodeMVCCValue decodes an MVCCKey from its Pebble representation. +// +// NOTE: this function does not inline, so it is not suitable for performance +// critical code paths. Instead, callers that care about performance and would +// like to avoid function calls should manually call the two decoding functions. +// tryDecodeSimpleMVCCValue does inline, so callers can use it to avoid making +// any function calls when decoding an MVCCValue that is encoded with the simple +// encoding. +func DecodeMVCCValue(buf []byte) (MVCCValue, error) { + v, ok, err := tryDecodeSimpleMVCCValue(buf) + if ok || err != nil { + return v, err + } + return decodeExtendedMVCCValue(buf) +} + +// Static error definitions, to permit inlining. +var errMVCCValueMissingTag = errors.Errorf("invalid encoded mvcc value, missing tag") +var errMVCCValueMissingHeader = errors.Errorf("invalid encoded mvcc value, missing header") + +// tryDecodeSimpleMVCCValue attempts to decode an MVCCValue that is using the +// simple encoding. If successful, returns the decoded value and true. If the +// value was using the extended encoding, returns false, in which case the +// caller should call decodeExtendedMVCCValue. +//gcassert:inline +func tryDecodeSimpleMVCCValue(buf []byte) (MVCCValue, bool, error) { + if len(buf) == 0 { + // Tombstone with no header. + return MVCCValue{}, true, nil + } + if len(buf) <= tagPos { + return MVCCValue{}, false, errMVCCValueMissingTag + } + if buf[tagPos] != extendedEncodingSentinel { + // Simple encoding. The encoding is equivalent to the roachpb.Value + // encoding, so inflate it directly. No need to copy or slice. + return MVCCValue{Value: roachpb.Value{RawBytes: buf}}, true, nil + } + // Extended encoding. The caller should call decodeExtendedMVCCValue. + return MVCCValue{}, false, nil +} + +func decodeExtendedMVCCValue(buf []byte) (MVCCValue, error) { + headerLen := binary.BigEndian.Uint32(buf) + headerSize := extendedPreludeSize + headerLen + if len(buf) < int(headerSize) { + return MVCCValue{}, errMVCCValueMissingHeader + } + var header enginepb.MVCCValueHeader + // NOTE: we don't use protoutil to avoid passing header through an interface, + // which would cause a heap allocation and incur the cost of dynamic dispatch. + if err := header.Unmarshal(buf[extendedPreludeSize:headerSize]); err != nil { + return MVCCValue{}, errors.Wrapf(err, "unmarshaling MVCCValueHeader") + } + var v MVCCValue + v.LocalTimestamp = header.LocalTimestamp + v.Value.RawBytes = buf[headerSize:] + return v, nil +} diff --git a/pkg/storage/mvcc_value_test.go b/pkg/storage/mvcc_value_test.go new file mode 100644 index 000000000000..fc8ee3473009 --- /dev/null +++ b/pkg/storage/mvcc_value_test.go @@ -0,0 +1,236 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +import ( + "bytes" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestMVCCValueLocalTimestampNeeded(t *testing.T) { + defer leaktest.AfterTest(t)() + + ts0 := hlc.Timestamp{Logical: 0} + ts1 := hlc.Timestamp{Logical: 1} + ts2 := hlc.Timestamp{Logical: 2} + + testcases := map[string]struct { + localTs hlc.Timestamp + versionTs hlc.Timestamp + expect bool + }{ + "no local timestamp": {ts0, ts2, false}, + "smaller local timestamp": {ts1, ts2, true}, + "equal local timestamp": {ts2, ts2, false}, + "larger local timestamp": {ts2, ts1, false}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + mvccKey := MVCCKey{Timestamp: tc.versionTs} + mvccVal := MVCCValue{} + mvccVal.LocalTimestamp = hlc.ClockTimestamp(tc.localTs) + + require.Equal(t, tc.expect, mvccVal.LocalTimestampNeeded(mvccKey)) + }) + } +} + +func TestMVCCValueGetLocalTimestamp(t *testing.T) { + defer leaktest.AfterTest(t)() + + ts0 := hlc.Timestamp{Logical: 0} + ts1 := hlc.Timestamp{Logical: 1} + ts2 := hlc.Timestamp{Logical: 2} + ts2S := hlc.Timestamp{Logical: 2, Synthetic: true} + + testcases := map[string]struct { + localTs hlc.Timestamp + versionTs hlc.Timestamp + expect hlc.Timestamp + }{ + "no local timestamp": {ts0, ts2, ts2}, + "no local timestamp, version synthetic": {ts0, ts2S, hlc.MinTimestamp}, + "smaller local timestamp": {ts1, ts2, ts1}, + "equal local timestamp": {ts2, ts2, ts2}, + "larger local timestamp": {ts2, ts1, ts2}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + mvccKey := MVCCKey{Timestamp: tc.versionTs} + mvccVal := MVCCValue{} + mvccVal.LocalTimestamp = hlc.ClockTimestamp(tc.localTs) + + require.Equal(t, hlc.ClockTimestamp(tc.expect), mvccVal.GetLocalTimestamp(mvccKey)) + }) + } +} + +func TestMVCCValueFormat(t *testing.T) { + defer leaktest.AfterTest(t)() + + var strVal, intVal roachpb.Value + strVal.SetString("foo") + intVal.SetInt(17) + + valHeader := enginepb.MVCCValueHeader{} + valHeader.LocalTimestamp = hlc.ClockTimestamp{WallTime: 9} + + testcases := map[string]struct { + val MVCCValue + expect string + }{ + "tombstone": {val: MVCCValue{}, expect: "/"}, + "bytes": {val: MVCCValue{Value: strVal}, expect: "/BYTES/foo"}, + "int": {val: MVCCValue{Value: intVal}, expect: "/INT/17"}, + "header+tombstone": {val: MVCCValue{MVCCValueHeader: valHeader}, expect: "vheader{ localTs=0.000000009,0 } /"}, + "header+bytes": {val: MVCCValue{MVCCValueHeader: valHeader, Value: strVal}, expect: "vheader{ localTs=0.000000009,0 } /BYTES/foo"}, + "header+int": {val: MVCCValue{MVCCValueHeader: valHeader, Value: intVal}, expect: "vheader{ localTs=0.000000009,0 } /INT/17"}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + require.Equal(t, tc.expect, tc.val.String()) + }) + } +} + +func TestEncodeDecodeMVCCValue(t *testing.T) { + defer leaktest.AfterTest(t)() + + var strVal, intVal roachpb.Value + strVal.SetString("foo") + intVal.SetInt(17) + + valHeader := enginepb.MVCCValueHeader{} + valHeader.LocalTimestamp = hlc.ClockTimestamp{WallTime: 9} + + testcases := map[string]struct { + val MVCCValue + expect []byte + }{ + "tombstone": {val: MVCCValue{}, expect: nil}, + "bytes": {val: MVCCValue{Value: strVal}, expect: []byte{0x0, 0x0, 0x0, 0x0, 0x3, 0x66, 0x6f, 0x6f}}, + "int": {val: MVCCValue{Value: intVal}, expect: []byte{0x0, 0x0, 0x0, 0x0, 0x1, 0x22}}, + "header+tombstone": {val: MVCCValue{MVCCValueHeader: valHeader}, expect: []byte{0x0, 0x0, 0x0, 0x4, 0x65, 0xa, 0x2, 0x8, 0x9}}, + "header+bytes": {val: MVCCValue{MVCCValueHeader: valHeader, Value: strVal}, expect: []byte{0x0, 0x0, 0x0, 0x4, 0x65, 0xa, 0x2, 0x8, 0x9, 0x0, 0x0, 0x0, 0x0, 0x3, 0x66, 0x6f, 0x6f}}, + "header+int": {val: MVCCValue{MVCCValueHeader: valHeader, Value: intVal}, expect: []byte{0x0, 0x0, 0x0, 0x4, 0x65, 0xa, 0x2, 0x8, 0x9, 0x0, 0x0, 0x0, 0x0, 0x1, 0x22}}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + encSize := encodedMVCCValueSize(tc.val) + require.Equal(t, len(tc.expect), encSize) + + enc, err := EncodeMVCCValue(tc.val) + require.NoError(t, err) + require.Equal(t, tc.expect, enc) + + dec, err := DecodeMVCCValue(enc) + require.NoError(t, err) + if len(dec.Value.RawBytes) == 0 { + dec.Value.RawBytes = nil // normalize + } + require.Equal(t, tc.val, dec) + }) + } +} + +func TestDecodeMVCCValueErrors(t *testing.T) { + defer leaktest.AfterTest(t)() + + testcases := map[string]struct { + enc []byte + expect error + }{ + "missing tag": {enc: []byte{0x0}, expect: errMVCCValueMissingTag}, + "missing header": {enc: []byte{0x0, 0x0, 0x0, 0x1, extendedEncodingSentinel}, expect: errMVCCValueMissingHeader}, + } + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + dec, err := DecodeMVCCValue(tc.enc) + require.Equal(t, tc.expect, err) + require.Zero(t, dec) + }) + } +} + +var mvccValueBenchmarkConfigs = struct { + headers map[string]enginepb.MVCCValueHeader + values map[string]roachpb.Value +}{ + headers: map[string]enginepb.MVCCValueHeader{ + "empty": {}, + "local walltime": {LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652545}}, + "local walltime+logical": {LocalTimestamp: hlc.ClockTimestamp{WallTime: 1643550788737652545, Logical: 4096}}, + }, + values: map[string]roachpb.Value{ + "tombstone": {}, + "short": roachpb.MakeValueFromString("foo"), + "long": roachpb.MakeValueFromBytes(bytes.Repeat([]byte{1}, 4096)), + }, +} + +func BenchmarkEncodeMVCCValue(b *testing.B) { + cfg := mvccValueBenchmarkConfigs + for hDesc, h := range cfg.headers { + for vDesc, v := range cfg.values { + name := fmt.Sprintf("header=%s/value=%s", hDesc, vDesc) + mvccValue := MVCCValue{MVCCValueHeader: h, Value: v} + b.Run(name, func(b *testing.B) { + for i := 0; i < b.N; i++ { + res, err := EncodeMVCCValue(mvccValue) + if err != nil { // for performance + require.NoError(b, err) + } + _ = res + } + }) + } + } +} + +func BenchmarkDecodeMVCCValue(b *testing.B) { + cfg := mvccValueBenchmarkConfigs + for hDesc, h := range cfg.headers { + for vDesc, v := range cfg.values { + for _, inline := range []bool{false, true} { + name := fmt.Sprintf("header=%s/value=%s/inline=%t", hDesc, vDesc, inline) + mvccValue := MVCCValue{MVCCValueHeader: h, Value: v} + buf, err := EncodeMVCCValue(mvccValue) + require.NoError(b, err) + b.Run(name, func(b *testing.B) { + for i := 0; i < b.N; i++ { + var res MVCCValue + var err error + if inline { + var ok bool + res, ok, err = tryDecodeSimpleMVCCValue(buf) + if !ok && err == nil { + res, err = decodeExtendedMVCCValue(buf) + } + } else { + res, err = DecodeMVCCValue(buf) + } + if err != nil { // for performance + require.NoError(b, err) + } + _ = res + } + }) + } + } + } +} diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index d9dc4d265020..6dbc0fec1e64 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1246,10 +1246,22 @@ func (p *Pebble) Merge(key MVCCKey, value []byte) error { } // PutMVCC implements the Engine interface. -func (p *Pebble) PutMVCC(key MVCCKey, value []byte) error { +func (p *Pebble) PutMVCC(key MVCCKey, value MVCCValue) error { if key.Timestamp.IsEmpty() { panic("PutMVCC timestamp is empty") } + encValue, err := EncodeMVCCValue(value) + if err != nil { + return err + } + return p.put(key, encValue) +} + +// PutRawMVCC implements the Engine interface. +func (p *Pebble) PutRawMVCC(key MVCCKey, value []byte) error { + if key.Timestamp.IsEmpty() { + panic("PutRawMVCC timestamp is empty") + } return p.put(key, value) } @@ -2071,7 +2083,11 @@ func (p *pebbleReadOnly) Merge(key MVCCKey, value []byte) error { panic("not implemented") } -func (p *pebbleReadOnly) PutMVCC(key MVCCKey, value []byte) error { +func (p *pebbleReadOnly) PutMVCC(key MVCCKey, value MVCCValue) error { + panic("not implemented") +} + +func (p *pebbleReadOnly) PutRawMVCC(key MVCCKey, value []byte) error { panic("not implemented") } @@ -2286,7 +2302,6 @@ func pebbleExportToSst( break } - unsafeValue := iter.UnsafeValue() isNewKey := !options.ExportAllRevisions || !unsafeKey.Key.Equal(curKey) if trackKeyBoundary && options.ExportAllRevisions && isNewKey { curKey = append(curKey[:0], unsafeKey.Key...) @@ -2319,10 +2334,27 @@ func pebbleExportToSst( } } - // Skip tombstone (len=0) records when start time is zero (non-incremental) - // and we are not exporting all versions. - skipTombstones := !options.ExportAllRevisions && options.StartTS.IsEmpty() - if len(unsafeValue) > 0 || !skipTombstones { + unsafeValue := iter.UnsafeValue() + skip := false + if unsafeKey.IsValue() { + mvccValue, ok, err := tryDecodeSimpleMVCCValue(unsafeValue) + if !ok && err == nil { + mvccValue, err = decodeExtendedMVCCValue(unsafeValue) + } + if err != nil { + return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding mvcc value %s", unsafeKey) + } + + // Export only the inner roachpb.Value, not the MVCCValue header. + unsafeValue = mvccValue.Value.RawBytes + + // Skip tombstone records when start time is zero (non-incremental) + // and we are not exporting all versions. + skipTombstones := !options.ExportAllRevisions && options.StartTS.IsEmpty() + skip = skipTombstones && mvccValue.IsTombstone() + } + + if !skip { if err := rows.Count(unsafeKey.Key); err != nil { return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "decoding %s", unsafeKey) } @@ -2351,7 +2383,7 @@ func pebbleExportToSst( return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "adding key %s", unsafeKey) } } else { - if err := sstWriter.PutMVCC(unsafeKey, unsafeValue); err != nil { + if err := sstWriter.PutRawMVCC(unsafeKey, unsafeValue); err != nil { return roachpb.BulkOpSummary{}, MVCCKey{}, errors.Wrapf(err, "adding key %s", unsafeKey) } } diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index ffb853472074..2ac471bc2188 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -438,10 +438,22 @@ func (p *pebbleBatch) Merge(key MVCCKey, value []byte) error { } // PutMVCC implements the Batch interface. -func (p *pebbleBatch) PutMVCC(key MVCCKey, value []byte) error { +func (p *pebbleBatch) PutMVCC(key MVCCKey, value MVCCValue) error { if key.Timestamp.IsEmpty() { panic("PutMVCC timestamp is empty") } + encValue, err := EncodeMVCCValue(value) + if err != nil { + return err + } + return p.put(key, encValue) +} + +// PutRawMVCC implements the Batch interface. +func (p *pebbleBatch) PutRawMVCC(key MVCCKey, value []byte) error { + if key.Timestamp.IsEmpty() { + panic("PutRawMVCC timestamp is empty") + } return p.put(key, value) } diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index f1897da83e14..94c2d9e1a959 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -345,11 +345,12 @@ type pebbleMVCCScanner struct { // cur* variables store the "current" record we're pointing to. Updated in // updateCurrent. Note that the timestamp can be clobbered in the case of // adding an intent from the intent history but is otherwise meaningful. - curUnsafeKey MVCCKey - curRawKey []byte - curValue []byte - results pebbleResults - intents pebble.Batch + curUnsafeKey MVCCKey + curRawKey []byte + curUnsafeValue MVCCValue + curRawValue []byte + results pebbleResults + intents pebble.Batch // mostRecentTS stores the largest timestamp observed that is equal to or // above the scan timestamp. Only applicable if failOnMoreRecent is true. If // set and no other error is hit, a WriteToOld error will be returned from @@ -548,11 +549,19 @@ func (p *pebbleMVCCScanner) uncertaintyError(ts hlc.Timestamp) bool { // continue. func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { if !p.curUnsafeKey.Timestamp.IsEmpty() { + if extended, valid := p.tryDecodeCurrentValueSimple(); !valid { + return false + } else if extended { + if !p.decodeCurrentValueExtended() { + return false + } + } + // ts < read_ts if p.curUnsafeKey.Timestamp.Less(p.ts) { // 1. Fast path: there is no intent and our read timestamp is newer // than the most recent version's timestamp. - return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curUnsafeValue.Value.RawBytes) } // ts == read_ts @@ -573,7 +582,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // 3. There is no intent and our read timestamp is equal to the most // recent version's timestamp. - return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curUnsafeValue.Value.RawBytes) } // ts > read_ts @@ -595,7 +604,8 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // 5. Our txn's read timestamp is less than the max timestamp // seen by the txn. We need to check for clock uncertainty // errors. - if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp) { + localTS := p.curUnsafeValue.GetLocalTimestamp(p.curUnsafeKey) + if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp, localTS) { return p.uncertaintyError(p.curUnsafeKey.Timestamp) } @@ -613,13 +623,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { return p.seekVersion(ctx, p.ts, false) } - if len(p.curValue) == 0 { - p.err = errors.Errorf("zero-length mvcc metadata") - return false - } - err := protoutil.Unmarshal(p.curValue, &p.meta) - if err != nil { - p.err = errors.Wrap(err, "unable to decode MVCCMetadata") + if !p.decodeCurrentMetadata() { return false } if len(p.meta.RawBytes) != 0 { @@ -675,11 +679,11 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // p.intents is a pebble.Batch which grows its byte slice capacity in // chunks to amortize allocations. The memMonitor is under-counting here // by only accounting for the key and value bytes. - if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curValue))); p.err != nil { + if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curRawValue))); p.err != nil { p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) return false } - p.err = p.intents.Set(p.curRawKey, p.curValue, nil) + p.err = p.intents.Set(p.curRawKey, p.curRawValue, nil) if p.err != nil { return false } @@ -703,11 +707,11 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // p.intents is a pebble.Batch which grows its byte slice capacity in // chunks to amortize allocations. The memMonitor is under-counting here // by only accounting for the key and value bytes. - if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curValue))); p.err != nil { + if p.err = p.memAccount.Grow(ctx, int64(len(p.curRawKey)+len(p.curRawValue))); p.err != nil { p.err = errors.Wrapf(p.err, "scan with start key %s", p.start) return false } - p.err = p.intents.Set(p.curRawKey, p.curValue, nil) + p.err = p.intents.Set(p.curRawKey, p.curRawValue, nil) if p.err != nil { return false } @@ -899,19 +903,19 @@ func (p *pebbleMVCCScanner) advanceKeyAtNewKey(key []byte) bool { // p.tombstones is true. Advances to the next key unless we've reached the max // results limit. func (p *pebbleMVCCScanner) addAndAdvance( - ctx context.Context, key roachpb.Key, rawKey []byte, val []byte, + ctx context.Context, key roachpb.Key, rawKey []byte, rawValue []byte, ) bool { // Don't include deleted versions len(val) == 0, unless we've been instructed // to include tombstones in the results. - if len(val) == 0 && !p.tombstones { + if len(rawValue) == 0 && !p.tombstones { return p.advanceKey() } // Check if adding the key would exceed a limit. if p.targetBytes > 0 && (p.results.bytes >= p.targetBytes || (p.targetBytesAvoidExcess && - p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(val))) > p.targetBytes)) { + p.results.bytes+int64(p.results.sizeOf(len(rawKey), len(rawValue))) > p.targetBytes)) { p.resumeReason = roachpb.RESUME_BYTE_LIMIT - p.resumeNextBytes = int64(p.results.sizeOf(len(rawKey), len(val))) + p.resumeNextBytes = int64(p.results.sizeOf(len(rawKey), len(rawValue))) } else if p.maxKeys > 0 && p.results.count >= p.maxKeys { p.resumeReason = roachpb.RESUME_KEY_LIMIT @@ -943,7 +947,7 @@ func (p *pebbleMVCCScanner) addAndAdvance( } } - if err := p.results.put(ctx, rawKey, val, p.memAccount); err != nil { + if err := p.results.put(ctx, rawKey, rawValue, p.memAccount); err != nil { p.err = errors.Wrapf(err, "scan with start key %s", p.start) return false } @@ -999,8 +1003,15 @@ func (p *pebbleMVCCScanner) seekVersion( } if p.curUnsafeKey.Timestamp.LessEq(seekTS) { p.incrementItersBeforeSeek() + if extended, valid := p.tryDecodeCurrentValueSimple(); !valid { + return false + } else if extended { + if !p.decodeCurrentValueExtended() { + return false + } + } if !uncertaintyCheck || p.curUnsafeKey.Timestamp.LessEq(p.ts) { - return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curUnsafeValue.Value.RawBytes) } // Iterate through uncertainty interval. Though we found a value in // the interval, it may not be uncertainty. This is because seekTS @@ -1010,7 +1021,8 @@ func (p *pebbleMVCCScanner) seekVersion( // are only uncertain if their timestamps are synthetic. Meanwhile, // any value with a time in the range (ts, uncertainty.LocalLimit] // is uncertain. - if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp) { + localTS := p.curUnsafeValue.GetLocalTimestamp(p.curUnsafeKey) + if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp, localTS) { return p.uncertaintyError(p.curUnsafeKey.Timestamp) } } @@ -1024,13 +1036,21 @@ func (p *pebbleMVCCScanner) seekVersion( if !bytes.Equal(p.curUnsafeKey.Key, origKey) { return p.advanceKeyAtNewKey(origKey) } + if extended, valid := p.tryDecodeCurrentValueSimple(); !valid { + return false + } else if extended { + if !p.decodeCurrentValueExtended() { + return false + } + } if !uncertaintyCheck || p.curUnsafeKey.Timestamp.LessEq(p.ts) { - return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curValue) + return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.curRawKey, p.curUnsafeValue.Value.RawBytes) } // Iterate through uncertainty interval. See the comment above about why // a value in this interval is not necessarily cause for an uncertainty // error. - if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp) { + localTS := p.curUnsafeValue.GetLocalTimestamp(p.curUnsafeKey) + if p.uncertainty.IsUncertain(p.curUnsafeKey.Timestamp, localTS) { return p.uncertaintyError(p.curUnsafeKey.Timestamp) } if !p.iterNext() { @@ -1039,7 +1059,9 @@ func (p *pebbleMVCCScanner) seekVersion( } } -// Updates cur{RawKey, Key, TS} to match record the iterator is pointing to. +// Updates cur{RawKey, UnsafeKey, RawValue} to match record the iterator is +// pointing to. Callers should call decodeCurrent{Metadata, Value} to decode +// the raw value if they need it. func (p *pebbleMVCCScanner) updateCurrent() bool { if !p.iterValid() { return false @@ -1050,12 +1072,45 @@ func (p *pebbleMVCCScanner) updateCurrent() bool { var err error p.curUnsafeKey, err = DecodeMVCCKey(p.curRawKey) if err != nil { - panic(err) + p.err = errors.Wrap(err, "unable to decode MVCCKey") + return false + } + p.curRawValue = p.parent.UnsafeValue() + + // Reset decoded value to avoid bugs. + if util.RaceEnabled { + p.meta = enginepb.MVCCMetadata{} + p.curUnsafeValue = MVCCValue{} } - p.curValue = p.parent.UnsafeValue() return true } +func (p *pebbleMVCCScanner) decodeCurrentMetadata() bool { + if len(p.curRawValue) == 0 { + p.err = errors.Errorf("zero-length mvcc metadata") + return false + } + err := protoutil.Unmarshal(p.curRawValue, &p.meta) + if err != nil { + p.err = errors.Wrap(err, "unable to decode MVCCMetadata") + return false + } + return true +} + +//gcassert:inline +func (p *pebbleMVCCScanner) tryDecodeCurrentValueSimple() (extended, valid bool) { + var simple bool + p.curUnsafeValue, simple, p.err = tryDecodeSimpleMVCCValue(p.curRawValue) + return !simple, p.err == nil +} + +//gcassert:inline +func (p *pebbleMVCCScanner) decodeCurrentValueExtended() bool { + p.curUnsafeValue, p.err = decodeExtendedMVCCValue(p.curRawValue) + return p.err == nil +} + func (p *pebbleMVCCScanner) iterValid() bool { if valid, err := p.parent.Valid(); !valid { // Defensive: unclear if p.err can already be non-nil here, but @@ -1138,9 +1193,9 @@ func (p *pebbleMVCCScanner) iterPeekPrev() ([]byte, bool) { // curRawKey, curKey and curValue to point to this saved data. We use a // single buffer for this purpose: savedBuf. p.savedBuf = append(p.savedBuf[:0], p.curRawKey...) - p.savedBuf = append(p.savedBuf, p.curValue...) + p.savedBuf = append(p.savedBuf, p.curRawValue...) p.curRawKey = p.savedBuf[:len(p.curRawKey)] - p.curValue = p.savedBuf[len(p.curRawKey):] + p.curRawValue = p.savedBuf[len(p.curRawKey):] // The raw key is always a prefix of the encoded MVCC key. Take advantage of this to // sub-slice the raw key directly, instead of calling SplitMVCCKey. p.curUnsafeKey.Key = p.curRawKey[:len(p.curUnsafeKey.Key)] diff --git a/pkg/storage/pebble_mvcc_scanner_test.go b/pkg/storage/pebble_mvcc_scanner_test.go index 597eb9a21c7a..93ca0bf53890 100644 --- a/pkg/storage/pebble_mvcc_scanner_test.go +++ b/pkg/storage/pebble_mvcc_scanner_test.go @@ -42,9 +42,10 @@ func TestMVCCScanWithManyVersionsAndSeparatedIntents(t *testing.T) { keys := []roachpb.Key{roachpb.Key("a"), roachpb.Key("b"), roachpb.Key("c")} // Many versions of each key. for i := 1; i < 10; i++ { + mvccValue := MVCCValue{Value: roachpb.MakeValueFromString(fmt.Sprintf("%d", i))} for _, k := range keys { - require.NoError(t, eng.PutMVCC(MVCCKey{Key: k, Timestamp: hlc.Timestamp{WallTime: int64(i)}}, - []byte(fmt.Sprintf("%d", i)))) + mvccKey := MVCCKey{Key: k, Timestamp: hlc.Timestamp{WallTime: int64(i)}} + require.NoError(t, eng.PutMVCC(mvccKey, mvccValue)) } } // Write a separated lock for the latest version of each key, to make it provisional. @@ -114,7 +115,7 @@ func TestMVCCScanWithManyVersionsAndSeparatedIntents(t *testing.T) { expectedKVs := make([]kv, len(keys)) for i := range expectedKVs { expectedKVs[i].k = MVCCKey{Key: keys[i], Timestamp: hlc.Timestamp{WallTime: 2}} - expectedKVs[i].v = []byte("2") + expectedKVs[i].v = roachpb.MakeValueFromString("2").RawBytes } require.Equal(t, expectedKVs, kvs) } @@ -129,13 +130,13 @@ func TestMVCCScanWithLargeKeyValue(t *testing.T) { largeValue := bytes.Repeat([]byte("l"), 150<<20) // Alternate small and large values. require.NoError(t, eng.PutMVCC(MVCCKey{Key: keys[0], Timestamp: hlc.Timestamp{WallTime: 1}}, - []byte("a"))) + MVCCValue{Value: roachpb.MakeValueFromBytes([]byte("a"))})) require.NoError(t, eng.PutMVCC(MVCCKey{Key: keys[1], Timestamp: hlc.Timestamp{WallTime: 1}}, - largeValue)) + MVCCValue{Value: roachpb.MakeValueFromBytes(largeValue)})) require.NoError(t, eng.PutMVCC(MVCCKey{Key: keys[2], Timestamp: hlc.Timestamp{WallTime: 1}}, - []byte("c"))) + MVCCValue{Value: roachpb.MakeValueFromBytes([]byte("c"))})) require.NoError(t, eng.PutMVCC(MVCCKey{Key: keys[3], Timestamp: hlc.Timestamp{WallTime: 1}}, - largeValue)) + MVCCValue{Value: roachpb.MakeValueFromBytes(largeValue)})) reader := eng.NewReadOnly(StandardDurability) defer reader.Close() @@ -159,14 +160,14 @@ func TestMVCCScanWithLargeKeyValue(t *testing.T) { numKeys := mvccScanner.results.count require.Equal(t, 4, int(numKeys)) require.Equal(t, 4, len(kvData)) - require.Equal(t, 20, len(kvData[0])) + require.Equal(t, 25, len(kvData[0])) require.Equal(t, 32, cap(kvData[0])) - require.Equal(t, 157286419, len(kvData[1])) - require.Equal(t, 157286419, cap(kvData[1])) - require.Equal(t, 20, len(kvData[2])) + require.Equal(t, 157286424, len(kvData[1])) + require.Equal(t, 157286424, cap(kvData[1])) + require.Equal(t, 25, len(kvData[2])) require.Equal(t, 32, cap(kvData[2])) - require.Equal(t, 157286419, len(kvData[3])) - require.Equal(t, 157286419, cap(kvData[3])) + require.Equal(t, 157286424, len(kvData[3])) + require.Equal(t, 157286424, cap(kvData[3])) } func scannerWithAccount( diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index c1e95b8a93d7..0fcf6fac653e 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -165,8 +165,9 @@ func TestPebbleIterReuse(t *testing.T) { batch := eng.NewBatch() defer batch.Close() for i := 0; i < 100; i++ { - key := MVCCKey{[]byte{byte(i)}, hlc.Timestamp{WallTime: 100}} - if err := batch.PutMVCC(key, []byte("foo")); err != nil { + key := MVCCKey{Key: []byte{byte(i)}, Timestamp: hlc.Timestamp{WallTime: 100}} + value := MVCCValue{Value: roachpb.MakeValueFromString("foo")} + if err := batch.PutMVCC(key, value); err != nil { t.Fatal(err) } } @@ -496,8 +497,9 @@ func TestPebbleIterConsistency(t *testing.T) { defer eng.Close() ts1 := hlc.Timestamp{WallTime: 1} ts2 := hlc.Timestamp{WallTime: 2} - k1 := MVCCKey{[]byte("a"), ts1} - require.NoError(t, eng.PutMVCC(k1, []byte("a1"))) + k1 := MVCCKey{Key: []byte("a"), Timestamp: ts1} + v1 := MVCCValue{Value: roachpb.MakeValueFromString("a1")} + require.NoError(t, eng.PutMVCC(k1, v1)) var ( roEngine = eng.NewReadOnly(StandardDurability) @@ -527,7 +529,9 @@ func TestPebbleIterConsistency(t *testing.T) { require.Nil(t, batch2.PinEngineStateForIterators()) // Write a newer version of "a" - require.NoError(t, eng.PutMVCC(MVCCKey{[]byte("a"), ts2}, []byte("a2"))) + k2 := MVCCKey{Key: []byte("a"), Timestamp: ts2} + v2 := MVCCValue{Value: roachpb.MakeValueFromString("a2")} + require.NoError(t, eng.PutMVCC(k2, v2)) checkMVCCIter := func(iter MVCCIterator) { defer iter.Close() @@ -594,8 +598,14 @@ func TestPebbleIterConsistency(t *testing.T) { // The eng iterator will see both values. checkIterSeesBothValues(eng.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: []byte("b")})) // The indexed batches will see 2 values since the second one is written to the batch. - require.NoError(t, batch.PutMVCC(MVCCKey{[]byte("a"), ts2}, []byte("a2"))) - require.NoError(t, batch2.PutMVCC(MVCCKey{[]byte("a"), ts2}, []byte("a2"))) + require.NoError(t, batch.PutMVCC( + MVCCKey{Key: []byte("a"), Timestamp: ts2}, + MVCCValue{Value: roachpb.MakeValueFromString("a2")}, + )) + require.NoError(t, batch2.PutMVCC( + MVCCKey{Key: []byte("a"), Timestamp: ts2}, + MVCCValue{Value: roachpb.MakeValueFromString("a2")}, + )) checkIterSeesBothValues(batch.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: []byte("b")})) checkIterSeesBothValues(batch2.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: []byte("b")})) } @@ -881,7 +891,10 @@ func TestPebbleBackgroundError(t *testing.T) { require.NoError(t, err) defer eng.Close() - require.NoError(t, eng.PutMVCC(MVCCKey{[]byte("a"), hlc.Timestamp{WallTime: 1}}, []byte("a"))) + require.NoError(t, eng.PutMVCC( + MVCCKey{Key: []byte("a"), Timestamp: hlc.Timestamp{WallTime: 1}}, + MVCCValue{Value: roachpb.MakeValueFromString("a")}, + )) require.NoError(t, eng.db.Flush()) } @@ -950,7 +963,8 @@ func generateData(t *testing.T, engine Engine, limits dataLimits, totalEntries i if rng.Float64() < limits.tombstoneChance { size = 0 } - require.NoError(t, engine.PutMVCC(MVCCKey{Key: key, Timestamp: timestamp}, randutil.RandBytes(rng, size)), "Write data to test storage") + value := MVCCValue{Value: roachpb.MakeValueFromBytes(randutil.RandBytes(rng, size))} + require.NoError(t, engine.PutMVCC(MVCCKey{Key: key, Timestamp: timestamp}, value), "Write data to test storage") } require.NoError(t, engine.Flush(), "Flush engine data") } @@ -1174,8 +1188,8 @@ func TestPebbleMVCCTimeIntervalCollectorAndFilter(t *testing.T) { aKey := roachpb.Key("a") for i := 0; i < 10; i++ { require.NoError(t, eng.PutMVCC( - MVCCKey{aKey, hlc.Timestamp{WallTime: int64(i), Logical: 1}}, - []byte(fmt.Sprintf("val%d", i)))) + MVCCKey{Key: aKey, Timestamp: hlc.Timestamp{WallTime: int64(i), Logical: 1}}, + MVCCValue{Value: roachpb.MakeValueFromString(fmt.Sprintf("val%d", i))})) } require.NoError(t, eng.Flush()) iter := eng.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ @@ -1205,9 +1219,10 @@ func TestPebbleFlushCallbackAndDurabilityRequirement(t *testing.T) { defer eng.Close() ts := hlc.Timestamp{WallTime: 1} - k := MVCCKey{[]byte("a"), ts} + k := MVCCKey{Key: []byte("a"), Timestamp: ts} + v := MVCCValue{Value: roachpb.MakeValueFromString("a1")} // Write. - require.NoError(t, eng.PutMVCC(k, []byte("a1"))) + require.NoError(t, eng.PutMVCC(k, v)) cbCount := int32(0) eng.RegisterFlushCompletedCallback(func() { atomic.AddInt32(&cbCount, 1) @@ -1234,7 +1249,7 @@ func TestPebbleFlushCallbackAndDurabilityRequirement(t *testing.T) { } return v } - require.Equal(t, "a1", string(checkGetAndIter(roStandard))) + require.Equal(t, v.Value.RawBytes, checkGetAndIter(roStandard)) // Write is not visible yet. require.Nil(t, checkGetAndIter(roGuaranteed)) require.Nil(t, checkGetAndIter(roGuaranteedPinned)) @@ -1251,5 +1266,5 @@ func TestPebbleFlushCallbackAndDurabilityRequirement(t *testing.T) { // due to iterator caching. roGuaranteed2 := eng.NewReadOnly(GuaranteedDurability) defer roGuaranteed2.Close() - require.Equal(t, "a1", string(checkGetAndIter(roGuaranteed2))) + require.Equal(t, v.Value.RawBytes, checkGetAndIter(roGuaranteed2)) } diff --git a/pkg/storage/sst.go b/pkg/storage/sst.go index b75253c4a066..35672f4b1fca 100644 --- a/pkg/storage/sst.go +++ b/pkg/storage/sst.go @@ -277,7 +277,7 @@ func UpdateSSTTimestamps( return nil, errors.Errorf("unexpected timestamp %s (expected %s) for key %s", key.Timestamp, from, key.Key) } - err = writer.PutMVCC(MVCCKey{Key: iter.UnsafeKey().Key, Timestamp: to}, iter.UnsafeValue()) + err = writer.PutRawMVCC(MVCCKey{Key: key.Key, Timestamp: to}, iter.UnsafeValue()) if err != nil { return nil, err } diff --git a/pkg/storage/sst_test.go b/pkg/storage/sst_test.go index fc467713ce3f..25fe889d27f1 100644 --- a/pkg/storage/sst_test.go +++ b/pkg/storage/sst_test.go @@ -71,7 +71,9 @@ func TestCheckSSTConflictsMaxIntents(t *testing.T) { // Write some committed keys and intents at txn1TS. batch := engine.NewBatch() for _, key := range keys { - require.NoError(t, batch.PutMVCC(MVCCKey{Key: roachpb.Key(key), Timestamp: txn1TS}, []byte("value"))) + mvccKey := MVCCKey{Key: roachpb.Key(key), Timestamp: txn1TS} + mvccValue := MVCCValue{Value: roachpb.MakeValueFromString("value")} + require.NoError(t, batch.PutMVCC(mvccKey, mvccValue)) } for _, key := range intents { require.NoError(t, MVCCPut(ctx, batch, nil, roachpb.Key(key), txn1TS, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("intent"), txn1)) @@ -146,11 +148,11 @@ func BenchmarkUpdateSSTTimestamps(b *testing.B) { b.Fatalf("unknown value mode %d", valueMode) } - var v roachpb.Value - v.SetBytes(value) - v.InitChecksum(key) + var v MVCCValue + v.Value.SetBytes(value) + v.Value.InitChecksum(key) - require.NoError(b, writer.PutMVCC(MVCCKey{Key: key, Timestamp: sstTimestamp}, v.RawBytes)) + require.NoError(b, writer.PutMVCC(MVCCKey{Key: key, Timestamp: sstTimestamp}, v)) } writer.Close() b.Logf("%vMB %v keys", sstFile.Len()/1e6, i) diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 0cff9a55897a..a7dac4766068 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -165,10 +165,25 @@ func (fw *SSTWriter) Put(key MVCCKey, value []byte) error { // An error is returned if it is not greater than any previously added entry // (according to the comparator configured during writer creation). `Close` // cannot have been called. -func (fw *SSTWriter) PutMVCC(key MVCCKey, value []byte) error { +func (fw *SSTWriter) PutMVCC(key MVCCKey, value MVCCValue) error { if key.Timestamp.IsEmpty() { panic("PutMVCC timestamp is empty") } + encValue, err := EncodeMVCCValue(value) + if err != nil { + return err + } + return fw.put(key, encValue) +} + +// PutRawMVCC implements the Writer interface. +// An error is returned if it is not greater than any previously added entry +// (according to the comparator configured during writer creation). `Close` +// cannot have been called. +func (fw *SSTWriter) PutRawMVCC(key MVCCKey, value []byte) error { + if key.Timestamp.IsEmpty() { + panic("PutRawMVCC timestamp is empty") + } return fw.put(key, value) } diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated index 065a36bdebbe..7ad2343a69dd 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated +++ b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_enable_separated @@ -51,7 +51,7 @@ with t=A txn_remove ---- >> at end: -data: "k"/124.000000000,0 -> /BYTES/v3 +data: "k"/124.000000000,0 -> vheader{ localTs=123.000000000,0 } /BYTES/v3 # Write value4 with an old timestamp without txn...should get a write # too old error. @@ -61,7 +61,7 @@ cput k=k v=v4 cond=v3 ts=123 ---- >> at end: data: "k"/124.000000000,1 -> /BYTES/v4 -data: "k"/124.000000000,0 -> /BYTES/v3 +data: "k"/124.000000000,0 -> vheader{ localTs=123.000000000,0 } /BYTES/v3 error: (*roachpb.WriteTooOldError:) WriteTooOldError: write for key "k" at timestamp 123.000000000,0 too old; wrote at 124.000000000,1 # Reset for next test diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums index 00173382f477..7bdadbd56c8c 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums @@ -161,12 +161,12 @@ with t=A meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "k" -> /BYTES/b @11.000000000,0 get: "k" -> /BYTES/b @11.000000000,0 -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=19 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false get: "k" -> /BYTES/b @14.000000000,0 >> at end: txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=32} lock=true stat=PENDING rts=11.000000000,0 wto=false gul=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false -data: "k"/14.000000000,0 -> /BYTES/b +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=19 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -174,7 +174,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 -# Ensure that the deleted value doens't surface. Instead, if we ignore the +# Ensure that the deleted value doesn't surface. Instead, if we ignore the # now-newest seq, we get the write before it. run ok @@ -225,11 +225,11 @@ with t=B txn_begin ts=20 get k=k ---- -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=19 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false get: "k" -> /BYTES/b @14.000000000,0 >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -255,7 +255,7 @@ meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 mi get: "l" -> /BYTES/c @20.000000000,0 >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -282,7 +282,7 @@ meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 mi get: "l" -> >> at end: txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=35} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -320,7 +320,7 @@ meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 mi get: "m" -> /BYTES/c @30.000000000,0 >> at end: txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=30.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -342,7 +342,7 @@ meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 mi get: "m" -> /BYTES/a @30.000000000,0 >> at end: txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=30.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -370,7 +370,7 @@ meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 mi get: "n" -> /BYTES/c @40.000000000,0 >> at end: txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=40.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -394,11 +394,11 @@ with t=D ---- meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "n" -> /BYTES/c @40.000000000,0 -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=20 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "n" -> /BYTES/c @45.000000000,0 >> at end: txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=40.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -406,8 +406,8 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false -data: "n"/45.000000000,0 -> /BYTES/c +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=20 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c # Ignore sequence numbers other than the current one, then commit. The value # shouldn't change. @@ -421,12 +421,12 @@ with t=E txn_begin ts=50 get k=n ---- -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=20 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false get: "n" -> /BYTES/c @45.000000000,0 get: "n" -> /BYTES/c @45.000000000,0 >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -434,7 +434,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c # Write a couple values at different sequence numbers on this key, then ignore # them all, then do a resolve_intent while the txn is pending. The intent should @@ -458,7 +458,7 @@ get: "n" -> /BYTES/c @45.000000000,0 get: "o" -> >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -468,7 +468,7 @@ data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "n"/50.000000000,0 -> /BYTES/c -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/c @@ -485,7 +485,7 @@ get: "n" -> /BYTES/c @45.000000000,0 get: "o" -> >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -493,7 +493,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c run ok with t=E @@ -533,7 +533,7 @@ with t=E meta: "o" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -541,7 +541,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/c @@ -556,7 +556,7 @@ with t=E get: "o" -> >> at end: txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=55.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -564,7 +564,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c run error @@ -589,7 +589,7 @@ meta: "o" -> txn={id=00000000 key="o" pri=0.00000000 epo=0 ts=50.000000000,0 min get: "o" -> /BYTES/b @50.000000000,0 >> at end: txn: "F" meta={id=00000000 key="o" pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=40.000000000,0 wto=false gul=0,0 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -597,7 +597,7 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c meta: "o"/0,0 -> txn={id=00000000 key="o" pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=20} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/b @@ -618,7 +618,7 @@ meta: "o" -> txn={id=00000000 key="o" pri=0.00000000 epo=0 ts=50.000000000,0 min get: "o" -> /BYTES/a @50.000000000,0 >> at end: txn: "F" meta={id=00000000 key="o" pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=40.000000000,0 wto=false gul=0,0 isn=1 -data: "k"/14.000000000,0 -> /BYTES/b +data: "k"/14.000000000,0 -> vheader{ localTs=11.000000000,0 } /BYTES/b meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/10"/11.000000000,0 -> /BYTES/10 meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true @@ -626,6 +626,6 @@ data: "k/20"/11.000000000,0 -> /BYTES/20 meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 mergeTs= txnDidNotUpdateMeta=true data: "k/30"/11.000000000,0 -> /BYTES/30 data: "m"/30.000000000,0 -> /BYTES/a -data: "n"/45.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> vheader{ localTs=40.000000000,0 } /BYTES/c meta: "o"/0,0 -> txn={id=00000000 key="o" pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=10} ts=50.000000000,0 del=false klen=12 vlen=6 mergeTs= txnDidNotUpdateMeta=false data: "o"/50.000000000,0 -> /BYTES/a diff --git a/pkg/storage/testdata/mvcc_histories/local_timestamp b/pkg/storage/testdata/mvcc_histories/local_timestamp new file mode 100644 index 000000000000..1f2670f66d1e --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/local_timestamp @@ -0,0 +1,324 @@ +# Exercise each write operation with a local timestamp. + +run ok +with ts=20 + put localTs=10 k=k1 v=v + put localTs=20 k=k2 v=v + put localTs=30 k=k3 v=v +---- +>> at end: +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/20.000000000,0 -> /BYTES/v + +run ok +with ts=30 + cput localTs=40 k=k1 v=v cond=v + cput localTs=30 k=k2 v=v cond=v + cput localTs=20 k=k3 v=v cond=v +---- +>> at end: +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +data: "k2"/30.000000000,0 -> /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v +data: "k3"/20.000000000,0 -> /BYTES/v + +run ok +with ts=40 + del localTs=30 k=k1 + del localTs=40 k=k2 + del localTs=50 k=k3 +---- +>> at end: +data: "k1"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } / +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +data: "k2"/40.000000000,0 -> / +data: "k2"/30.000000000,0 -> /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/40.000000000,0 -> / +data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v +data: "k3"/20.000000000,0 -> /BYTES/v + +run ok +with ts=50 + increment localTs=60 k=k1 + increment localTs=50 k=k2 + increment localTs=40 k=k3 +---- +inc: current value = 1 +inc: current value = 1 +inc: current value = 1 +>> at end: +data: "k1"/50.000000000,0 -> /INT/1 +data: "k1"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } / +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +data: "k2"/50.000000000,0 -> /INT/1 +data: "k2"/40.000000000,0 -> / +data: "k2"/30.000000000,0 -> /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/50.000000000,0 -> vheader{ localTs=40.000000000,0 } /INT/1 +data: "k3"/40.000000000,0 -> / +data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v +data: "k3"/20.000000000,0 -> /BYTES/v + +run ok +with ts=60 + del_range localTs=50 k=k1 end=k2 + del_range localTs=60 k=k2 end=k3 + del_range localTs=70 k=k3 end=k4 +---- +del_range: "k1"-"k2" -> deleted 1 key(s) +del_range: "k2"-"k3" -> deleted 1 key(s) +del_range: "k3"-"k4" -> deleted 1 key(s) +>> at end: +data: "k1"/60.000000000,0 -> vheader{ localTs=50.000000000,0 } / +data: "k1"/50.000000000,0 -> /INT/1 +data: "k1"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } / +data: "k1"/30.000000000,0 -> /BYTES/v +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +data: "k2"/60.000000000,0 -> / +data: "k2"/50.000000000,0 -> /INT/1 +data: "k2"/40.000000000,0 -> / +data: "k2"/30.000000000,0 -> /BYTES/v +data: "k2"/20.000000000,0 -> /BYTES/v +data: "k3"/60.000000000,0 -> / +data: "k3"/50.000000000,0 -> vheader{ localTs=40.000000000,0 } /INT/1 +data: "k3"/40.000000000,0 -> / +data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v +data: "k3"/20.000000000,0 -> /BYTES/v + +run ok +with ts=19 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> +get: "k2" -> +get: "k3" -> + +run ok +with ts=20 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /BYTES/v @20.000000000,0 +get: "k2" -> /BYTES/v @20.000000000,0 +get: "k3" -> /BYTES/v @20.000000000,0 + +run ok +with ts=29 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /BYTES/v @20.000000000,0 +get: "k2" -> /BYTES/v @20.000000000,0 +get: "k3" -> /BYTES/v @20.000000000,0 + +run ok +with ts=30 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /BYTES/v @30.000000000,0 +get: "k2" -> /BYTES/v @30.000000000,0 +get: "k3" -> /BYTES/v @30.000000000,0 + +run ok +with ts=39 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /BYTES/v @30.000000000,0 +get: "k2" -> /BYTES/v @30.000000000,0 +get: "k3" -> /BYTES/v @30.000000000,0 + +run ok +with ts=40 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> +get: "k2" -> +get: "k3" -> + +run ok +with ts=49 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> +get: "k2" -> +get: "k3" -> + +run ok +with ts=50 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /INT/1 @50.000000000,0 +get: "k2" -> /INT/1 @50.000000000,0 +get: "k3" -> /INT/1 @50.000000000,0 + +run ok +with ts=59 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> /INT/1 @50.000000000,0 +get: "k2" -> /INT/1 @50.000000000,0 +get: "k3" -> /INT/1 @50.000000000,0 + +run ok +with ts=60 + get k=k1 + get k=k2 + get k=k3 +---- +get: "k1" -> +get: "k2" -> +get: "k3" -> + +run ok +clear_range k=k1 end=k4 +---- +>> at end: + + + +# Exercise intent resolution with and without prior observation timestamps. + +run ok +with t=A ts=20 localTs=10 + txn_begin + put k=k1 v=v + put k=k2 v=v + put k=k3 v=v + put k=k4 v=v + put k=k5 v=v + put k=k6 v=v + put k=k7 v=v + put k=k8 v=v + put k=k9 v=v + put k=k10 v=v + put k=k11 v=v + put k=k12 v=v +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k1"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k10"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k11"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k11"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k12"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k12"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k2"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k4"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k5"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k6"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k7"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k8"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k8"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v +meta: "k9"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=19 mergeTs= txnDidNotUpdateMeta=true +data: "k9"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v + +run ok +with t=A + txn_step + txn_advance ts=30 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 + +run ok +with t=A localTs=20 + put k=k1 v=v2 + put k=k2 v=v2 + put k=k3 v=v2 + put k=k4 v=v2 + put k=k5 v=v2 + put k=k6 v=v2 + put k=k7 v=v2 + put k=k8 v=v2 + put k=k9 v=v2 + put k=k10 v=v2 + put k=k11 v=v2 + put k=k12 v=v2 +---- +>> at end: +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k1"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k10"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k11"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k11"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k12"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k12"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k2"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k3"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k4"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k4"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k5"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k5"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k6"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k6"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k7"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k7"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k8"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k8"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k9"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=1} ts=30.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k9"/30.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 + +run ok +with t=A + txn_advance ts=40 + resolve_intent k=k1 status=ABORTED + resolve_intent k=k2 status=ABORTED clockWhilePending=20 + resolve_intent k=k3 status=ABORTED clockWhilePending=30 + resolve_intent k=k4 status=ABORTED clockWhilePending=40 + resolve_intent k=k5 status=PENDING + resolve_intent k=k6 status=PENDING clockWhilePending=20 + resolve_intent k=k7 status=PENDING clockWhilePending=30 + resolve_intent k=k8 status=PENDING clockWhilePending=40 + resolve_intent k=k9 status=COMMITTED + resolve_intent k=k10 status=COMMITTED clockWhilePending=20 + resolve_intent k=k11 status=COMMITTED clockWhilePending=30 + resolve_intent k=k12 status=COMMITTED clockWhilePending=40 +---- +>> at end: +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k10"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +data: "k11"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } /BYTES/v2 +data: "k12"/40.000000000,0 -> /BYTES/v2 +meta: "k5"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k5"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k6"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k6"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 +meta: "k7"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=20 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k7"/40.000000000,0 -> vheader{ localTs=30.000000000,0 } /BYTES/v2 +meta: "k8"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=1} ts=40.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} mergeTs= txnDidNotUpdateMeta=false +data: "k8"/40.000000000,0 -> /BYTES/v2 +data: "k9"/40.000000000,0 -> vheader{ localTs=20.000000000,0 } /BYTES/v2 diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit index 41f854262a02..94bee39a51b5 100644 --- a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit @@ -4,25 +4,25 @@ # k1: value @ ts 10 # # k2: value @ ts 20 -# k2: value @ ts 10? +# k2: value @ ts 10, localTs 5 # -# k3: value @ ts 20? +# k3: value @ ts 20, localTs 10 # k3: value @ ts 10 # -# k4: value @ ts 20? -# k4: value @ ts 10? +# k4: value @ ts 20, localTs 10 +# k4: value @ ts 10, localTs 5 # # k5: intent @ ts 20 # k5: value @ ts 10 # # k6: intent @ ts 20 -# k6: value @ ts 10? +# k6: value @ ts 10, localTs 5 # -# k7: intent @ ts 20? +# k7: intent @ ts 20, localTs 10 # k7: value @ ts 10 # -# k8: intent @ ts 20? -# k8: value @ ts 10? +# k8: intent @ ts 20, localTs 10 +# k8: value @ ts 10, localTs 5 # run ok @@ -36,42 +36,42 @@ data: "k1"/10.000000000,0 -> /BYTES/v1 run ok with k=k2 - put v=v3 ts=10,0? + put v=v3 ts=10,0 localTs=5,0 put v=v4 ts=20,0 ---- >> at end: data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 run ok with k=k3 put v=v5 ts=10,0 - put v=v6 ts=20,0? + put v=v6 ts=20,0 localTs=10,0 ---- >> at end: data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 run ok with k=k4 - put v=v7 ts=10,0? - put v=v8 ts=20,0? + put v=v7 ts=10,0 localTs=5,0 + put v=v8 ts=20,0 localTs=10,0 ---- >> at end: data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 -data: "k4"/20.000000000,0? -> /BYTES/v8 -data: "k4"/10.000000000,0? -> /BYTES/v7 +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v8 +data: "k4"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v7 run ok with k=k5 @@ -84,18 +84,18 @@ txn: "A" meta={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0 data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 -data: "k4"/20.000000000,0? -> /BYTES/v8 -data: "k4"/10.000000000,0? -> /BYTES/v7 +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v8 +data: "k4"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v7 meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 run ok with k=k6 - put v=v11 ts=10,0? + put v=v11 ts=10,0 localTs=5,0 txn_begin t=B ts=20,0 put t=B v=v12 ---- @@ -104,72 +104,72 @@ txn: "B" meta={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0 data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 -data: "k4"/20.000000000,0? -> /BYTES/v8 -data: "k4"/10.000000000,0? -> /BYTES/v7 +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v8 +data: "k4"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v7 meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k6"/20.000000000,0 -> /BYTES/v12 -data: "k6"/10.000000000,0? -> /BYTES/v11 +data: "k6"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v11 run ok with k=k7 put v=v13 ts=10,0 - txn_begin t=C ts=20,0? - put t=C v=v14 + txn_begin t=C ts=20,0 + put t=C v=v14 localTs=10,0 ---- >> at end: -txn: "C" meta={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0? wto=false gul=0,0 +txn: "C" meta={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 -data: "k4"/20.000000000,0? -> /BYTES/v8 -data: "k4"/10.000000000,0? -> /BYTES/v7 +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v8 +data: "k4"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v7 meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k6"/20.000000000,0 -> /BYTES/v12 -data: "k6"/10.000000000,0? -> /BYTES/v11 -meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true -data: "k7"/20.000000000,0? -> /BYTES/v14 +data: "k6"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=21 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v14 data: "k7"/10.000000000,0 -> /BYTES/v13 run ok with k=k8 - put v=v15 ts=10,0? - txn_begin t=D ts=20,0? - put t=D v=v16 + put v=v15 ts=10,0 localTs=5,0 + txn_begin t=D ts=20,0 + put t=D v=v16 localTs=10,0 ---- >> at end: -txn: "D" meta={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0? wto=false gul=0,0 +txn: "D" meta={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 data: "k1"/20.000000000,0 -> /BYTES/v2 data: "k1"/10.000000000,0 -> /BYTES/v1 data: "k2"/20.000000000,0 -> /BYTES/v4 -data: "k2"/10.000000000,0? -> /BYTES/v3 -data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k2"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v3 +data: "k3"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v6 data: "k3"/10.000000000,0 -> /BYTES/v5 -data: "k4"/20.000000000,0? -> /BYTES/v8 -data: "k4"/10.000000000,0? -> /BYTES/v7 +data: "k4"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v8 +data: "k4"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v7 meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k5"/20.000000000,0 -> /BYTES/v10 data: "k5"/10.000000000,0 -> /BYTES/v9 meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true data: "k6"/20.000000000,0 -> /BYTES/v12 -data: "k6"/10.000000000,0? -> /BYTES/v11 -meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true -data: "k7"/20.000000000,0? -> /BYTES/v14 +data: "k6"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=21 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v14 data: "k7"/10.000000000,0 -> /BYTES/v13 -meta: "k8"/0,0 -> txn={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true -data: "k8"/20.000000000,0? -> /BYTES/v16 -data: "k8"/10.000000000,0? -> /BYTES/v15 +meta: "k8"/0,0 -> txn={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=21 mergeTs= txnDidNotUpdateMeta=true +data: "k8"/20.000000000,0 -> vheader{ localTs=10.000000000,0 } /BYTES/v16 +data: "k8"/10.000000000,0 -> vheader{ localTs=5.000000000,0 } /BYTES/v15 # Test cases: # @@ -204,13 +204,13 @@ run error get t=txn1 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k3 localUncertaintyLimit=5,0 @@ -226,13 +226,13 @@ run error get t=txn1 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k5 localUncertaintyLimit=5,0 @@ -248,13 +248,13 @@ run error get t=txn1 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok get t=txn1 k=k7 localUncertaintyLimit=5,0 @@ -270,13 +270,13 @@ run error get t=txn1 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run error scan t=txn1 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] run ok @@ -299,13 +299,13 @@ run error get t=txn2 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k3 localUncertaintyLimit=5,0 @@ -321,13 +321,13 @@ run error get t=txn2 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k5 localUncertaintyLimit=5,0 @@ -343,13 +343,13 @@ run error get t=txn2 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok get t=txn2 k=k7 localUncertaintyLimit=5,0 @@ -365,13 +365,13 @@ run error get t=txn2 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn2 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok @@ -394,37 +394,35 @@ run error get t=txn3 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] -run error +run ok get t=txn3 k=k3 localUncertaintyLimit=5,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] -run error +run ok scan t=txn3 k=k3 localUncertaintyLimit=5,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn3 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn3 k=k5 localUncertaintyLimit=5,0 @@ -440,37 +438,35 @@ run error get t=txn3 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] -run error +run ok get t=txn3 k=k7 localUncertaintyLimit=5,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] -run error +run ok scan t=txn3 k=k7 localUncertaintyLimit=5,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn3 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn3 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -493,37 +489,35 @@ run error get t=txn4 k=k2 localUncertaintyLimit=5,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k2 localUncertaintyLimit=5,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] -run error +run ok get t=txn4 k=k3 localUncertaintyLimit=5,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] -run error +run ok scan t=txn4 k=k3 localUncertaintyLimit=5,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn4 k=k4 localUncertaintyLimit=5,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k4 localUncertaintyLimit=5,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn4 k=k5 localUncertaintyLimit=5,0 @@ -539,37 +533,35 @@ run error get t=txn4 k=k6 localUncertaintyLimit=5,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k6 localUncertaintyLimit=5,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] -run error +run ok get t=txn4 k=k7 localUncertaintyLimit=5,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] -run error +run ok scan t=txn4 k=k7 localUncertaintyLimit=5,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn4 k=k8 localUncertaintyLimit=5,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn4 k=k8 localUncertaintyLimit=5,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -594,13 +586,13 @@ run error get t=txn5 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k3 localUncertaintyLimit=10,0 @@ -618,13 +610,13 @@ run error get t=txn5 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k5 localUncertaintyLimit=10,0 @@ -642,13 +634,13 @@ run error get t=txn5 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error get t=txn5 k=k7 localUncertaintyLimit=10,0 @@ -666,13 +658,13 @@ run error get t=txn5 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run error scan t=txn5 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] run ok @@ -697,37 +689,37 @@ run error get t=txn6 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k5 localUncertaintyLimit=10,0 @@ -745,37 +737,37 @@ run error get t=txn6 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn6 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn6 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -800,37 +792,37 @@ run error get t=txn7 k=k2 localUncertaintyLimit=10,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k2 localUncertaintyLimit=10,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k5 localUncertaintyLimit=10,0 @@ -848,37 +840,37 @@ run error get t=txn7 k=k6 localUncertaintyLimit=10,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k6 localUncertaintyLimit=10,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn7 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn7 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -903,37 +895,37 @@ run error get t=txn8 k=k2 localUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k2 localUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k5 localUncertaintyLimit=15,0 @@ -951,37 +943,37 @@ run error get t=txn8 k=k6 localUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k6 localUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn8 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn8 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1006,37 +998,37 @@ run error get t=txn9 k=k2 localUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k2 localUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k5 localUncertaintyLimit=15,0 @@ -1054,37 +1046,37 @@ run error get t=txn9 k=k6 localUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k6 localUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn9 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn9 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1121,25 +1113,25 @@ run error get t=txn10 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k5 localUncertaintyLimit=20,0 @@ -1169,25 +1161,25 @@ run error get t=txn10 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn10 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn10 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1209,12 +1201,12 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn11 k=k2 localUncertaintyLimit=10,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn11 k=k2 localUncertaintyLimit=10,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run ok get t=txn11 k=k3 localUncertaintyLimit=10,0 @@ -1229,12 +1221,12 @@ scan: "k3" -> /BYTES/v5 @10.000000000,0 run ok get t=txn11 k=k4 localUncertaintyLimit=10,0 ---- -get: "k4" -> /BYTES/v7 @10.000000000,0? +get: "k4" -> /BYTES/v7 @10.000000000,0 run ok scan t=txn11 k=k4 localUncertaintyLimit=10,0 ---- -scan: "k4" -> /BYTES/v7 @10.000000000,0? +scan: "k4" -> /BYTES/v7 @10.000000000,0 run ok get t=txn11 k=k5 localUncertaintyLimit=10,0 @@ -1249,12 +1241,12 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn11 k=k6 localUncertaintyLimit=10,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn11 k=k6 localUncertaintyLimit=10,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run ok get t=txn11 k=k7 localUncertaintyLimit=10,0 @@ -1269,12 +1261,12 @@ scan: "k7" -> /BYTES/v13 @10.000000000,0 run ok get t=txn11 k=k8 localUncertaintyLimit=10,0 ---- -get: "k8" -> /BYTES/v15 @10.000000000,0? +get: "k8" -> /BYTES/v15 @10.000000000,0 run ok scan t=txn11 k=k8 localUncertaintyLimit=10,0 ---- -scan: "k8" -> /BYTES/v15 @10.000000000,0? +scan: "k8" -> /BYTES/v15 @10.000000000,0 run ok @@ -1296,36 +1288,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn12 k=k2 localUncertaintyLimit=10,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn12 k=k2 localUncertaintyLimit=10,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn12 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn12 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn12 k=k5 localUncertaintyLimit=10,0 @@ -1340,36 +1332,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn12 k=k6 localUncertaintyLimit=10,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn12 k=k6 localUncertaintyLimit=10,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn12 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn12 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn12 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1391,36 +1383,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn13 k=k2 localUncertaintyLimit=10,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn13 k=k2 localUncertaintyLimit=10,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn13 k=k3 localUncertaintyLimit=10,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k3 localUncertaintyLimit=10,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn13 k=k4 localUncertaintyLimit=10,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k4 localUncertaintyLimit=10,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn13 k=k5 localUncertaintyLimit=10,0 @@ -1435,36 +1427,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn13 k=k6 localUncertaintyLimit=10,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn13 k=k6 localUncertaintyLimit=10,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn13 k=k7 localUncertaintyLimit=10,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k7 localUncertaintyLimit=10,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn13 k=k8 localUncertaintyLimit=10,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn13 k=k8 localUncertaintyLimit=10,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1486,36 +1478,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn14 k=k2 localUncertaintyLimit=15,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn14 k=k2 localUncertaintyLimit=15,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn14 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn14 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn14 k=k5 localUncertaintyLimit=15,0 @@ -1530,36 +1522,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn14 k=k6 localUncertaintyLimit=15,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn14 k=k6 localUncertaintyLimit=15,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn14 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn14 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn14 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1581,36 +1573,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn15 k=k2 localUncertaintyLimit=15,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn15 k=k2 localUncertaintyLimit=15,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn15 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn15 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn15 k=k5 localUncertaintyLimit=15,0 @@ -1625,36 +1617,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn15 k=k6 localUncertaintyLimit=15,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn15 k=k6 localUncertaintyLimit=15,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn15 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn15 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn15 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1691,25 +1683,25 @@ run error get t=txn16 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k5 localUncertaintyLimit=20,0 @@ -1739,25 +1731,25 @@ run error get t=txn16 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn16 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn16 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1779,36 +1771,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn17 k=k2 localUncertaintyLimit=15,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn17 k=k2 localUncertaintyLimit=15,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn17 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn17 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok get t=txn17 k=k5 localUncertaintyLimit=15,0 @@ -1823,36 +1815,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn17 k=k6 localUncertaintyLimit=15,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn17 k=k6 localUncertaintyLimit=15,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn17 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error get t=txn17 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run error scan t=txn17 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] run ok @@ -1874,36 +1866,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get t=txn18 k=k2 localUncertaintyLimit=15,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn18 k=k2 localUncertaintyLimit=15,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get t=txn18 k=k3 localUncertaintyLimit=15,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k3 localUncertaintyLimit=15,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn18 k=k4 localUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k4 localUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok get t=txn18 k=k5 localUncertaintyLimit=15,0 @@ -1918,36 +1910,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get t=txn18 k=k6 localUncertaintyLimit=15,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan t=txn18 k=k6 localUncertaintyLimit=15,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get t=txn18 k=k7 localUncertaintyLimit=15,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k7 localUncertaintyLimit=15,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn18 k=k8 localUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn18 k=k8 localUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -1984,25 +1976,25 @@ run error get t=txn19 k=k3 localUncertaintyLimit=20,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k3 localUncertaintyLimit=20,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k4 localUncertaintyLimit=20,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k4 localUncertaintyLimit=20,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k5 localUncertaintyLimit=20,0 @@ -2032,25 +2024,25 @@ run error get t=txn19 k=k7 localUncertaintyLimit=20,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k7 localUncertaintyLimit=20,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error get t=txn19 k=k8 localUncertaintyLimit=20,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run error scan t=txn19 k=k8 localUncertaintyLimit=20,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] run ok @@ -2082,22 +2074,22 @@ scan: "k2" -> /BYTES/v4 @20.000000000,0 run ok get t=txn20 k=k3 localUncertaintyLimit=20,0 ---- -get: "k3" -> /BYTES/v6 @20.000000000,0? +get: "k3" -> /BYTES/v6 @20.000000000,0 run ok scan t=txn20 k=k3 localUncertaintyLimit=20,0 ---- -scan: "k3" -> /BYTES/v6 @20.000000000,0? +scan: "k3" -> /BYTES/v6 @20.000000000,0 run ok get t=txn20 k=k4 localUncertaintyLimit=20,0 ---- -get: "k4" -> /BYTES/v8 @20.000000000,0? +get: "k4" -> /BYTES/v8 @20.000000000,0 run ok scan t=txn20 k=k4 localUncertaintyLimit=20,0 ---- -scan: "k4" -> /BYTES/v8 @20.000000000,0? +scan: "k4" -> /BYTES/v8 @20.000000000,0 run error get t=txn20 k=k5 localUncertaintyLimit=20,0 @@ -2254,13 +2246,13 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2276,13 +2268,13 @@ run error get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2298,13 +2290,13 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 @@ -2320,13 +2312,13 @@ run error get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 @@ -2342,37 +2334,35 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] -run error +run ok get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] -run error +run ok scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 @@ -2388,37 +2378,35 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] -run error +run ok get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] -run error +run ok scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2436,13 +2424,13 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2460,13 +2448,13 @@ run error get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2484,13 +2472,13 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2508,13 +2496,13 @@ run error get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 @@ -2532,37 +2520,37 @@ run error get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 @@ -2580,37 +2568,37 @@ run error get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k6" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k6"-"k6\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2640,25 +2628,25 @@ run error get k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2688,25 +2676,25 @@ run error get k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2721,12 +2709,12 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run ok get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2741,12 +2729,12 @@ scan: "k3" -> /BYTES/v5 @10.000000000,0 run ok get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -get: "k4" -> /BYTES/v7 @10.000000000,0? +get: "k4" -> /BYTES/v7 @10.000000000,0 run ok scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -scan: "k4" -> /BYTES/v7 @10.000000000,0? +scan: "k4" -> /BYTES/v7 @10.000000000,0 run ok get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2761,12 +2749,12 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run ok get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 @@ -2781,12 +2769,12 @@ scan: "k7" -> /BYTES/v13 @10.000000000,0 run ok get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -get: "k8" -> /BYTES/v15 @10.000000000,0? +get: "k8" -> /BYTES/v15 @10.000000000,0 run ok scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 ---- -scan: "k8" -> /BYTES/v15 @10.000000000,0? +scan: "k8" -> /BYTES/v15 @10.000000000,0 run ok get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 @@ -2801,36 +2789,36 @@ scan: "k1" -> /BYTES/v1 @10.000000000,0 run ok get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- -get: "k2" -> /BYTES/v3 @10.000000000,0? +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- -scan: "k2" -> /BYTES/v3 @10.000000000,0? +scan: "k2" -> /BYTES/v3 @10.000000000,0 run error get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 @@ -2845,36 +2833,36 @@ scan: "k5" -> /BYTES/v9 @10.000000000,0 run ok get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- -get: "k6" -> /BYTES/v11 @10.000000000,0? +get: "k6" -> /BYTES/v11 @10.000000000,0 run ok scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- -scan: "k6" -> /BYTES/v11 @10.000000000,0? +scan: "k6" -> /BYTES/v11 @10.000000000,0 run error get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2904,25 +2892,25 @@ run error get k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k3" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k3"-"k3\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k4" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k4"-"k4\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2952,25 +2940,25 @@ run error get k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k7" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k7"-"k7\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error get k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- get: "k8" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run error scan k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- scan: "k8"-"k8\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] run ok get k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 @@ -2995,22 +2983,22 @@ scan: "k2" -> /BYTES/v4 @20.000000000,0 run ok get k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- -get: "k3" -> /BYTES/v6 @20.000000000,0? +get: "k3" -> /BYTES/v6 @20.000000000,0 run ok scan k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- -scan: "k3" -> /BYTES/v6 @20.000000000,0? +scan: "k3" -> /BYTES/v6 @20.000000000,0 run ok get k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- -get: "k4" -> /BYTES/v8 @20.000000000,0? +get: "k4" -> /BYTES/v8 @20.000000000,0 run ok scan k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 ---- -scan: "k4" -> /BYTES/v8 @20.000000000,0? +scan: "k4" -> /BYTES/v8 @20.000000000,0 run error get k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps new file mode 100644 index 000000000000..41f854262a02 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_and_synthetic_timestamps @@ -0,0 +1,3061 @@ +# Setup: +# +# k1: value @ ts 20 +# k1: value @ ts 10 +# +# k2: value @ ts 20 +# k2: value @ ts 10? +# +# k3: value @ ts 20? +# k3: value @ ts 10 +# +# k4: value @ ts 20? +# k4: value @ ts 10? +# +# k5: intent @ ts 20 +# k5: value @ ts 10 +# +# k6: intent @ ts 20 +# k6: value @ ts 10? +# +# k7: intent @ ts 20? +# k7: value @ ts 10 +# +# k8: intent @ ts 20? +# k8: value @ ts 10? +# + +run ok +with k=k1 + put v=v1 ts=10,0 + put v=v2 ts=20,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 + +run ok +with k=k2 + put v=v3 ts=10,0? + put v=v4 ts=20,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 + +run ok +with k=k3 + put v=v5 ts=10,0 + put v=v6 ts=20,0? +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 + +run ok +with k=k4 + put v=v7 ts=10,0? + put v=v8 ts=20,0? +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0? -> /BYTES/v8 +data: "k4"/10.000000000,0? -> /BYTES/v7 + +run ok +with k=k5 + put v=v9 ts=10,0 + txn_begin t=A ts=20,0 + put t=A v=v10 +---- +>> at end: +txn: "A" meta={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0? -> /BYTES/v8 +data: "k4"/10.000000000,0? -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 + +run ok +with k=k6 + put v=v11 ts=10,0? + txn_begin t=B ts=20,0 + put t=B v=v12 +---- +>> at end: +txn: "B" meta={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0? -> /BYTES/v8 +data: "k4"/10.000000000,0? -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0? -> /BYTES/v11 + +run ok +with k=k7 + put v=v13 ts=10,0 + txn_begin t=C ts=20,0? + put t=C v=v14 +---- +>> at end: +txn: "C" meta={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0? wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0? -> /BYTES/v8 +data: "k4"/10.000000000,0? -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0? -> /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0? -> /BYTES/v14 +data: "k7"/10.000000000,0 -> /BYTES/v13 + +run ok +with k=k8 + put v=v15 ts=10,0? + txn_begin t=D ts=20,0? + put t=D v=v16 +---- +>> at end: +txn: "D" meta={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0? wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0? -> /BYTES/v3 +data: "k3"/20.000000000,0? -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0? -> /BYTES/v8 +data: "k4"/10.000000000,0? -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0? -> /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0? -> /BYTES/v14 +data: "k7"/10.000000000,0 -> /BYTES/v13 +meta: "k8"/0,0 -> txn={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0? min=0,0 seq=0} ts=20.000000000,0? del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k8"/20.000000000,0? -> /BYTES/v16 +data: "k8"/10.000000000,0? -> /BYTES/v15 + +# Test cases: +# +# for ts in (5, 10, 15, 20, 25): +# for localUncertaintyLimit in (5, 10, 15, 20, 25): +# if localUncertaintyLimit < ts: continue +# for globalUncertaintyLimit in (5, 10, 15, 20, 25): +# if globalUncertaintyLimit < ts: continue +# if globalUncertaintyLimit <= localUncertaintyLimit: continue +# for k in (k1, k2, k3, k4, k5, k6, k7, k8): +# for op in (get, scan): +# testCase() +# + +run ok +txn_begin t=txn1 ts=5,0 globalUncertaintyLimit=10,0 +---- +>> at end: +txn: "txn1" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=10.000000000,0 + +run ok +get t=txn1 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn1 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get t=txn1 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run error +scan t=txn1 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run ok +get t=txn1 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn1 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run error +get t=txn1 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run error +scan t=txn1 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run ok +get t=txn1 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn1 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get t=txn1 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run error +scan t=txn1 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run ok +get t=txn1 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn1 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run error +get t=txn1 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + +run error +scan t=txn1 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=10.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn2 ts=5,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn2" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=15.000000000,0 + +run ok +get t=txn2 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn2 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get t=txn2 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn2 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run ok +get t=txn2 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn2 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run error +get t=txn2 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn2 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run ok +get t=txn2 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn2 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get t=txn2 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn2 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run ok +get t=txn2 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn2 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run error +get t=txn2 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn2 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=15.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn3 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn3" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn3 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn3 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get t=txn3 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn3 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn3 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run ok +get t=txn3 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn3 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get t=txn3 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn3 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn3 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn3 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn4 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn4" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn4 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn4 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get t=txn4 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn4 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn4 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run ok +get t=txn4 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn4 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get t=txn4 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn4 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn4 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn4 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn5 ts=5,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn5" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=15.000000000,0 + +run error +get t=txn5 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn6 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn6" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run error +get t=txn6 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn7 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn7" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn7 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn8 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn8" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run error +get t=txn8 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn9 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn9" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn9 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn10 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn10" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn10 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn11 ts=10,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn11" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=15.000000000,0 + +run ok +get t=txn11 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn11 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn11 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn11 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +get t=txn11 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn11 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn11 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0? + +run ok +scan t=txn11 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0? + +run ok +get t=txn11 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn11 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn11 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn11 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +get t=txn11 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn11 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn11 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0? + +run ok +scan t=txn11 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0? + + +run ok +txn_begin t=txn12 ts=10,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn12" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn12 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn12 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn12 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn12 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn12 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn12 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn12 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn12 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run ok +get t=txn12 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn12 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn12 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn12 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn12 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn12 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn12 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn12 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn13 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn13" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn13 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn13 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn13 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn13 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn13 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn13 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn13 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn13 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run ok +get t=txn13 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn13 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn13 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn13 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn13 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn13 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn13 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn13 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn14 ts=10,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn14" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn14 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn14 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn14 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn14 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn14 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn14 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn14 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn14 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run ok +get t=txn14 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn14 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn14 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn14 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn14 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn14 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn14 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn14 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn15 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn15" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn15 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn15 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn15 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn15 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn15 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn15 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn15 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn15 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run ok +get t=txn15 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn15 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn15 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn15 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn15 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn15 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn15 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn15 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn16 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn16" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn16 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn17 ts=15,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn17" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn17 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn17 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn17 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn17 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn17 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn17 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn17 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn17 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run ok +get t=txn17 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn17 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn17 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn17 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn17 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn17 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn17 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn17 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn18 ts=15,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn18" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn18 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn18 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn18 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan t=txn18 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get t=txn18 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn18 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn18 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn18 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run ok +get t=txn18 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn18 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn18 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan t=txn18 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get t=txn18 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn18 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn18 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn18 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn19 ts=15,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn19" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn19 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn20 ts=20,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn20" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn20 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +scan t=txn20 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +get t=txn20 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +scan t=txn20 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +get t=txn20 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> /BYTES/v6 @20.000000000,0? + +run ok +scan t=txn20 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3" -> /BYTES/v6 @20.000000000,0? + +run ok +get t=txn20 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> /BYTES/v8 @20.000000000,0? + +run ok +scan t=txn20 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4" -> /BYTES/v8 @20.000000000,0? + +run error +get t=txn20 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +scan t=txn20 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +get t=txn20 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +scan t=txn20 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +get t=txn20 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +scan t=txn20 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +get t=txn20 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +run error +scan t=txn20 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +# A subset of the previous test cases, but with non-transactional reads: +# +# for ts in (5, 15, 25): +# for localUncertaintyLimit in (5, 15, 25): +# if localUncertaintyLimit < ts: continue +# for globalUncertaintyLimit in (5, 15, 25): +# if globalUncertaintyLimit < ts: continue +# if globalUncertaintyLimit < localUncertaintyLimit: continue +# for k in (k1, k2, k3, k4, k5, k6, k7, k8): +# for op in (get, scan): +# testCase() +# + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> + +run ok +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> + +run error +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> + +run ok +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> + +run error +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> + +run error +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> + +run error +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=5.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0? + +run ok +scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0? + +run ok +get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0? + +run ok +scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0? + +run ok +get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0? + +run ok +scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0? + +run error +get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0? + +run ok +scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0? + +run error +get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0? within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +scan k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +get k=k2 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +scan k=k2 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +get k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> /BYTES/v6 @20.000000000,0? + +run ok +scan k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3" -> /BYTES/v6 @20.000000000,0? + +run ok +get k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> /BYTES/v8 @20.000000000,0? + +run ok +scan k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4" -> /BYTES/v8 @20.000000000,0? + +run error +get k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +scan k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +get k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +scan k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +get k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +scan k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +get k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +run error +scan k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 642d3821fb1b..a337c0004427 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1052,6 +1052,7 @@ func TestLint(t *testing.T) { ":!rpc/codec.go", ":!rpc/codec_test.go", ":!settings/settings_test.go", + ":!storage/mvcc_value.go", ":!sql/types/types_jsonpb.go", ":!sql/schemachanger/scplan/internal/scgraphviz/graphviz.go", ) @@ -1096,6 +1097,7 @@ func TestLint(t *testing.T) { ":!util/hlc/timestamp.go", ":!rpc/codec.go", ":!rpc/codec_test.go", + ":!storage/mvcc_value.go", ":!sql/types/types_jsonpb.go", ) if err != nil { diff --git a/pkg/testutils/sstutil/kv.go b/pkg/testutils/sstutil/kv.go index a01e8b9f0805..ff2b8ecf16b6 100644 --- a/pkg/testutils/sstutil/kv.go +++ b/pkg/testutils/sstutil/kv.go @@ -51,7 +51,7 @@ func (kv KV) Value() roachpb.Value { return value } -// ValueBytes returns the roachpb.Value byte-representation of the value. -func (kv KV) ValueBytes() []byte { - return kv.Value().RawBytes +// MVCCValue returns the storage.MVCCValue representation of the value. +func (kv KV) MVCCValue() storage.MVCCValue { + return storage.MVCCValue{Value: kv.Value()} } diff --git a/pkg/testutils/sstutil/sstutil.go b/pkg/testutils/sstutil/sstutil.go index 8ab97c0e270f..a0e9cb202647 100644 --- a/pkg/testutils/sstutil/sstutil.go +++ b/pkg/testutils/sstutil/sstutil.go @@ -41,12 +41,12 @@ func MakeSST(t *testing.T, st *cluster.Settings, kvs []KV) ([]byte, roachpb.Key, end = kv.Key() } if kv.Timestamp().IsEmpty() { - meta := &enginepb.MVCCMetadata{RawBytes: kv.ValueBytes()} + meta := &enginepb.MVCCMetadata{RawBytes: kv.Value().RawBytes} metaBytes, err := protoutil.Marshal(meta) require.NoError(t, err) require.NoError(t, writer.PutUnversioned(kv.Key(), metaBytes)) } else { - require.NoError(t, writer.PutMVCC(kv.MVCCKey(), kv.ValueBytes())) + require.NoError(t, writer.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) } } require.NoError(t, writer.Finish()) @@ -73,8 +73,9 @@ func ScanSST(t *testing.T, sst []byte) []KV { } k := iter.UnsafeKey() - v := roachpb.Value{RawBytes: iter.UnsafeValue()} - value, err := v.GetBytes() + v, err := storage.DecodeMVCCValue(iter.UnsafeValue()) + require.NoError(t, err) + value, err := v.Value.GetBytes() require.NoError(t, err) kvs = append(kvs, KV{ KeyString: string(k.Key), diff --git a/pkg/util/hlc/doc.go b/pkg/util/hlc/doc.go index c6080bdd350a..733cc0cd75f6 100644 --- a/pkg/util/hlc/doc.go +++ b/pkg/util/hlc/doc.go @@ -70,15 +70,13 @@ The linearizability guarantee is important to note as two sequential (in real time) transactions via two different gateway nodes can be assigned timestamps in reverse order (the second gateway's clock may be behind), but must still see results according to real-time order if they access overlapping keys (e.g. B -must see A's write). Also keep in mind that an intent's written timestamp +must see A's write). Also keep in mind that an intent's local timestamp signifies when the intent itself was written, but the final value will be resolved to the transaction's commit timestamp, which may be later than the -written timestamp. Since the commit status and timestamp are non-local +local timestamp. Since the commit status and timestamp are non-local properties, a range may contain committed values (as unresolved intents) that turn out to exist in the future of the local HLC when the intent gets resolved. -TODO(nvanbenschoten): Update the above on written timestamps after #72121. - - Cooperative lease transfers (Raft channel). During a cooperative lease transfer from one replica of a range to another, the outgoing leaseholder revokes its lease before its expiration time and consults its clock to @@ -89,7 +87,7 @@ TODO(nvanbenschoten): Update the above on written timestamps after #72121. The invariant that a leaseholder's clock is always >= its lease's start time is used in a few places. First, it ensures that the leaseholder's clock - always leads the written_timestamp of any value in its keyspace written by a + always leads the local_timestamp of any value in its keyspace written by a prior leaseholder on its range, which is an important property for the correctness of observed timestamps. Second, it ensures that the leaseholder immediately views itself as the leaseholder. Third, it ensures that if the @@ -97,9 +95,6 @@ TODO(nvanbenschoten): Update the above on written timestamps after #72121. this later lease's start time could be pulled from the local clock and be guaranteed to receive an even greater starting timestamp. - TODO(nvanbenschoten): the written_timestamp concept does not yet exist in - code. It will be introduced in the replacement to #72121. - - Range merges (Raft + BatchRequest channels). During a merge of two ranges, the right-hand side of the merge passes a "frozen timestamp" clock reading from the right-hand side leaseholder, through the merge transaction @@ -109,21 +104,21 @@ TODO(nvanbenschoten): Update the above on written timestamps after #72121. merge and officially takes control of the combined range, it forwards its HLC to this frozen timestamp. Like the previous interaction, this one is also necessary to ensure that the leaseholder of the joint range has a clock that - leads the written_timestamp of any value in its keyspace, even one written + leads the local_timestamp of any value in its keyspace, even one written originally on the right-hand side range. - Observed timestamps (Raft + BatchRequest channels). During the lifetime of a transaction, its coordinator issues BatchRequests to other nodes in the cluster. Each time a given transaction visits a node for the first time, it captures an observation from the node's HLC. Separately, when a leaseholder - on a given node serves a write, it ensures that the node's HLC clock is >= - the written_timestamp of the write. This written_timestamp is retained even - if an intent is moved to a higher timestamp if it is asynchronously resolved. - As a result, these "observed timestamps" captured during the lifetime of a - transaction can be used to make a claim about values that could not have been - written yet at the time that the transaction first visited the node, and by - extension, at the time that the transaction began. This allows the - transaction to avoid uncertainty restarts in some circumstances. + on a given node serves a write, it assigns the write a local_timestamp from + its node's HLC clock. This local_timestamp is retained even if an intent is + moved to a higher timestamp if it is asynchronously resolved. As a result, + these "observed timestamps" captured during the lifetime of a transaction can + be used to make a claim about values that could not have been written yet at + the time that the transaction first visited the node, and by extension, at + the time that the transaction began. This allows the transaction to avoid + uncertainty restarts in some circumstances. A variant of this same mechanism applies to non-transactional requests that defer their timestamp allocation to the leaseholder of their (single) range.