diff --git a/pkg/cmd/roachtest/tests/acceptance.go b/pkg/cmd/roachtest/tests/acceptance.go index c0c45fb1efb1..d489c182be63 100644 --- a/pkg/cmd/roachtest/tests/acceptance.go +++ b/pkg/cmd/roachtest/tests/acceptance.go @@ -52,7 +52,6 @@ func registerAcceptance(r registry.Registry) { }, { name: "version-upgrade", - skip: "WIP: unskip when version checks are added to local_timestamp writes", fn: func(ctx context.Context, t test.Test, c cluster.Cluster) { runVersionUpgrade(ctx, t, c) }, diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index ae012cb87a87..636c05d33c7e 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -655,6 +655,10 @@ func (s spanSetWriter) LogLogicalOp( s.w.LogLogicalOp(op, details) } +func (s spanSetWriter) ShouldWriteLocalTimestamps(ctx context.Context) bool { + return s.w.ShouldWriteLocalTimestamps(ctx) +} + // ReadWriter is used outside of the spanset package internally, in ccl. type ReadWriter struct { spanSetReader diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 61780523cec9..39763785b4bf 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -645,6 +645,11 @@ type Writer interface { // // It is safe to modify the contents of the arguments after it returns. SingleClearEngineKey(key EngineKey) error + + // ShouldWriteLocalTimestamps is only for internal use in the storage package. + // This method is temporary, to handle the transition from clusters where not + // all nodes understand local timestamps. + ShouldWriteLocalTimestamps(ctx context.Context) bool } // ReadWriter is the read/write interface to an engine's data. diff --git a/pkg/storage/intent_reader_writer.go b/pkg/storage/intent_reader_writer.go index 3217153b8950..15b55297f4e2 100644 --- a/pkg/storage/intent_reader_writer.go +++ b/pkg/storage/intent_reader_writer.go @@ -30,9 +30,8 @@ type intentDemuxWriter struct { w Writer } -func wrapIntentWriter(ctx context.Context, w Writer) intentDemuxWriter { - idw := intentDemuxWriter{w: w} - return idw +func wrapIntentWriter(w Writer) intentDemuxWriter { + return intentDemuxWriter{w: w} } // ClearIntent has the same behavior as Writer.ClearIntent. buf is used as diff --git a/pkg/storage/intent_reader_writer_test.go b/pkg/storage/intent_reader_writer_test.go index baab535d195a..55bceaedfe2d 100644 --- a/pkg/storage/intent_reader_writer_test.go +++ b/pkg/storage/intent_reader_writer_test.go @@ -200,7 +200,7 @@ func TestIntentDemuxWriter(t *testing.T) { // This is a low-level test that explicitly wraps the writer, so it // doesn't matter how the original call to createTestPebbleEngine // behaved in terms of separated intents config. - w = wrapIntentWriter(context.Background(), &pw) + w = wrapIntentWriter(&pw) return "" case "put-intent": pw.reset() diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 76c7421460ca..6c237a175a50 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -1597,7 +1597,11 @@ func mvccPutInternal( } buf.newMeta.Txn = txnMeta buf.newMeta.Timestamp = writeTimestamp.ToLegacyTimestamp() - buf.newMeta.SetLocalTimestamp(localTimestamp, &buf.newLocalTs) + if writer.ShouldWriteLocalTimestamps(ctx) { + buf.newMeta.SetLocalTimestamp(localTimestamp, &buf.newLocalTs) + } else { + buf.newMeta.LocalTimestamp = nil + } } newMeta := &buf.newMeta @@ -3084,9 +3088,13 @@ func mvccResolveWriteIntent( // resolver provides a clock observation from this node that was captured // while the transaction was still pending, in which case it can be advanced // to the observed timestamp. - localTs := latestKey.LocalTimestamp - localTs.Forward(intent.ClockWhilePending.Timestamp) - buf.newMeta.SetLocalTimestamp(localTs, &buf.newLocalTs) + if rw.ShouldWriteLocalTimestamps(ctx) { + localTs := latestKey.LocalTimestamp + localTs.Forward(intent.ClockWhilePending.Timestamp) + buf.newMeta.SetLocalTimestamp(localTs, &buf.newLocalTs) + } else { + buf.newMeta.LocalTimestamp = nil + } // Update or remove the metadata key. var metaKeySize, metaValSize int64 diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 287b2056ec58..36e2a27c828d 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/uncertainty" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -99,18 +98,16 @@ func TestMVCCHistories(t *testing.T) { datadriven.Walk(t, testutils.TestDataPath(t, "mvcc_histories"), func(t *testing.T, path string) { // We start from a clean slate in every test file. - engine, err := Open(ctx, InMemory(), CacheSize(1<<20 /* 1 MiB */), - func(cfg *engineConfig) error { - // Latest cluster version, since these tests are not ones where we - // are examining differences related to separated intents. - cfg.Settings = cluster.MakeTestingClusterSettings() - return nil - }) + engine, err := Open(ctx, InMemory(), CacheSize(1<<20 /* 1 MiB */)) if err != nil { t.Fatal(err) } defer engine.Close() + if strings.Contains(path, "_disable_local_timestamps") { + localTimestampsEnabled.Override(ctx, &engine.settings.SV, false) + } + reportDataEntries := func(buf *redact.StringBuilder) error { hasData := false err := engine.MVCCIterate(span.Key, span.EndKey, MVCCKeyAndIntentsIterKind, func(r MVCCKeyValue) error { diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 361c484ebf9f..2f327de76569 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -931,7 +931,7 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) { p.makeMetricEtcEventListener(ctx), ) p.eventListener = &cfg.Opts.EventListener - p.wrappedIntentWriter = wrapIntentWriter(ctx, p) + p.wrappedIntentWriter = wrapIntentWriter(p) // Read the current store cluster version. storeClusterVersion, err := getMinVersion(unencryptedFS, cfg.Dir) @@ -1298,6 +1298,45 @@ func (p *Pebble) LogLogicalOp(op MVCCLogicalOpType, details MVCCLogicalOpDetails // No-op. Logical logging disabled. } +// localTimestampsEnabled controls whether local timestamps are written on MVCC +// keys. A true setting is also gated on clusterversion.TODO. After all nodes in +// a cluster are at or beyond clusterversion.TODO, different nodes will see the +// version state transition at different times. Nodes that have not yet seen the +// transition may remove the local timestamp from an intent that has one during +// intent resolution. This will not cause problems. +// +// TODO(nvanbenschoten): remove this cluster setting and its associated plumbing +// when removing the cluster version, once we're confident in the efficacy and +// stability of local timestamps. +var localTimestampsEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "storage.transaction.local_timestamps.enabled", + "if enabled, MVCC keys will be written with local timestamps", + true, +) + +func shouldWriteLocalTimestamps(ctx context.Context, settings *cluster.Settings) bool { + if !localTimestampsEnabled.Get(&settings.SV) { + // Not enabled. + return false + } + ver := settings.Version.ActiveVersionOrEmpty(ctx) + if ver == (clusterversion.ClusterVersion{}) { + // Some tests fail to configure settings. In these cases, assume that it + // is safe to write local timestamps. + return true + } + // TODO(nvanbenschoten): add a new cluster version when the time comes. + return ver.IsActive(clusterversion.SeedSpanCountTable) +} + +// ShouldWriteLocalTimestamps implements the Writer interface. +func (p *Pebble) ShouldWriteLocalTimestamps(ctx context.Context) bool { + // This is not fast. Pebble should not be used by writers that want + // performance. They should use pebbleBatch. + return shouldWriteLocalTimestamps(ctx, p.settings) +} + // Attrs implements the Engine interface. func (p *Pebble) Attrs() roachpb.Attributes { return p.attrs @@ -1534,7 +1573,7 @@ func (p *Pebble) GetAuxiliaryDir() string { // NewBatch implements the Engine interface. func (p *Pebble) NewBatch() Batch { - return newPebbleBatch(p.db, p.db.NewIndexedBatch(), false /* writeOnly */) + return newPebbleBatch(p.db, p.db.NewIndexedBatch(), false /* writeOnly */, p.settings) } // NewReadOnly implements the Engine interface. @@ -1544,7 +1583,7 @@ func (p *Pebble) NewReadOnly(durability DurabilityRequirement) ReadWriter { // NewUnindexedBatch implements the Engine interface. func (p *Pebble) NewUnindexedBatch(writeOnly bool) Batch { - return newPebbleBatch(p.db, p.db.NewBatch(), writeOnly) + return newPebbleBatch(p.db, p.db.NewBatch(), writeOnly, p.settings) } // NewSnapshot implements the Engine interface. @@ -2104,6 +2143,10 @@ func (p *pebbleReadOnly) LogLogicalOp(op MVCCLogicalOpType, details MVCCLogicalO panic("not implemented") } +func (p *pebbleReadOnly) ShouldWriteLocalTimestamps(ctx context.Context) bool { + panic("not implemented") +} + // pebbleSnapshot represents a snapshot created using Pebble.NewSnapshot(). type pebbleSnapshot struct { snapshot *pebble.Snapshot diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index ffb853472074..906db648e274 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -16,6 +16,7 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -56,6 +57,8 @@ type pebbleBatch struct { wrappedIntentWriter intentDemuxWriter // scratch space for wrappedIntentWriter. scratch []byte + + shouldWriteLocalTimestamps bool } var _ Batch = &pebbleBatch{} @@ -67,7 +70,9 @@ var pebbleBatchPool = sync.Pool{ } // Instantiates a new pebbleBatch. -func newPebbleBatch(db *pebble.DB, batch *pebble.Batch, writeOnly bool) *pebbleBatch { +func newPebbleBatch( + db *pebble.DB, batch *pebble.Batch, writeOnly bool, settings *cluster.Settings, +) *pebbleBatch { pb := pebbleBatchPool.Get().(*pebbleBatch) *pb = pebbleBatch{ db: db, @@ -94,8 +99,11 @@ func newPebbleBatch(db *pebble.DB, batch *pebble.Batch, writeOnly bool) *pebbleB reusable: true, }, writeOnly: writeOnly, + + // pebbleBatch is short-lived, so cache the value for performance. + shouldWriteLocalTimestamps: shouldWriteLocalTimestamps(context.Background(), settings), } - pb.wrappedIntentWriter = wrapIntentWriter(context.Background(), pb) + pb.wrappedIntentWriter = wrapIntentWriter(pb) return pb } @@ -528,3 +536,8 @@ func (p *pebbleBatch) Repr() []byte { copy(reprCopy, repr) return reprCopy } + +// ShouldWriteLocalTimestamps implements the Writer interface. +func (p *pebbleBatch) ShouldWriteLocalTimestamps(ctx context.Context) bool { + return p.shouldWriteLocalTimestamps +} diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 0cff9a55897a..bf0936f8c345 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -319,6 +319,11 @@ func (fw *SSTWriter) Close() { fw.fw = nil } +// ShouldWriteLocalTimestamps implements the Writer interface. +func (fw *SSTWriter) ShouldWriteLocalTimestamps(context.Context) bool { + return false +} + // MemFile is a file-like struct that buffers all data written to it in memory. // Implements the writeCloseSyncer interface and is intended for use with // SSTWriter. diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps new file mode 100644 index 000000000000..65f8dd368959 --- /dev/null +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval_with_local_uncertainty_limit_disable_local_timestamps @@ -0,0 +1,2945 @@ +# Setup: +# +# k1: value @ ts 20 +# k1: value @ ts 10 +# +# k2: value @ ts 20 +# k2: value @ ts 10@5 +# +# k3: value @ ts 20@10 +# k3: value @ ts 10 +# +# k4: value @ ts 20@10 +# k4: value @ ts 10@5 +# +# k5: intent @ ts 20 +# k5: value @ ts 10 +# +# k6: intent @ ts 20 +# k6: value @ ts 10@5 +# +# k7: intent @ ts 20@10 +# k7: value @ ts 10 +# +# k8: intent @ ts 20@10 +# k8: value @ ts 10@5 +# + +run ok +with k=k1 + put v=v1 ts=10,0 + put v=v2 ts=20,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 + +run ok +with k=k2 + put v=v3 ts=10,0 localTs=5,0 + put v=v4 ts=20,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 + +run ok +with k=k3 + put v=v5 ts=10,0 + put v=v6 ts=20,0 localTs=10,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 + +run ok +with k=k4 + put v=v7 ts=10,0 localTs=5,0 + put v=v8 ts=20,0 localTs=10,0 +---- +>> at end: +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0 -> /BYTES/v7 + +run ok +with k=k5 + put v=v9 ts=10,0 + txn_begin t=A ts=20,0 + put t=A v=v10 +---- +>> at end: +txn: "A" meta={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0 -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 + +run ok +with k=k6 + put v=v11 ts=10,0 localTs=5,0 + txn_begin t=B ts=20,0 + put t=B v=v12 +---- +>> at end: +txn: "B" meta={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0 -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0 -> /BYTES/v11 + +run ok +with k=k7 + put v=v13 ts=10,0 + txn_begin t=C ts=20,0 + put t=C v=v14 localTs=10,0 +---- +>> at end: +txn: "C" meta={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0 -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0 -> /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0 -> /BYTES/v14 +data: "k7"/10.000000000,0 -> /BYTES/v13 + +run ok +with k=k8 + put v=v15 ts=10,0 localTs=5,0 + txn_begin t=D ts=20,0 + put t=D v=v16 localTs=10,0 +---- +>> at end: +txn: "D" meta={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v1 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 +data: "k3"/20.000000000,0 -> /BYTES/v6 +data: "k3"/10.000000000,0 -> /BYTES/v5 +data: "k4"/20.000000000,0 -> /BYTES/v8 +data: "k4"/10.000000000,0 -> /BYTES/v7 +meta: "k5"/0,0 -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k5"/20.000000000,0 -> /BYTES/v10 +data: "k5"/10.000000000,0 -> /BYTES/v9 +meta: "k6"/0,0 -> txn={id=00000000 key="k6" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k6"/20.000000000,0 -> /BYTES/v12 +data: "k6"/10.000000000,0 -> /BYTES/v11 +meta: "k7"/0,0 -> txn={id=00000000 key="k7" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k7"/20.000000000,0 -> /BYTES/v14 +data: "k7"/10.000000000,0 -> /BYTES/v13 +meta: "k8"/0,0 -> txn={id=00000000 key="k8" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=8 mergeTs= txnDidNotUpdateMeta=true +data: "k8"/20.000000000,0 -> /BYTES/v16 +data: "k8"/10.000000000,0 -> /BYTES/v15 + +# Test cases: +# +# for ts in (5, 10, 15, 20, 25): +# for localUncertaintyLimit in (5, 10, 15, 20, 25): +# if localUncertaintyLimit < ts: continue +# for globalUncertaintyLimit in (5, 10, 15, 20, 25): +# if globalUncertaintyLimit < ts: continue +# if globalUncertaintyLimit <= localUncertaintyLimit: continue +# for k in (k1, k2, k3, k4, k5, k6, k7, k8): +# for op in (get, scan): +# testCase() +# + +run ok +txn_begin t=txn1 ts=5,0 globalUncertaintyLimit=10,0 +---- +>> at end: +txn: "txn1" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=10.000000000,0 + +run ok +get t=txn1 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn1 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get t=txn1 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan t=txn1 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get t=txn1 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn1 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get t=txn1 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan t=txn1 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get t=txn1 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn1 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get t=txn1 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan t=txn1 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get t=txn1 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn1 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get t=txn1 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan t=txn1 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + + +run ok +txn_begin t=txn2 ts=5,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn2" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=15.000000000,0 + +run ok +get t=txn2 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn2 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get t=txn2 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan t=txn2 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get t=txn2 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn2 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get t=txn2 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan t=txn2 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get t=txn2 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn2 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get t=txn2 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan t=txn2 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get t=txn2 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn2 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get t=txn2 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan t=txn2 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + + +run ok +txn_begin t=txn3 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn3" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn3 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn3 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get t=txn3 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan t=txn3 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get t=txn3 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn3 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get t=txn3 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan t=txn3 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get t=txn3 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn3 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get t=txn3 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan t=txn3 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get t=txn3 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn3 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get t=txn3 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan t=txn3 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + + +run ok +txn_begin t=txn4 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn4" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn4 k=k1 localUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan t=txn4 k=k1 localUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get t=txn4 k=k2 localUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan t=txn4 k=k2 localUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get t=txn4 k=k3 localUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan t=txn4 k=k3 localUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get t=txn4 k=k4 localUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan t=txn4 k=k4 localUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get t=txn4 k=k5 localUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan t=txn4 k=k5 localUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get t=txn4 k=k6 localUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan t=txn4 k=k6 localUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get t=txn4 k=k7 localUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan t=txn4 k=k7 localUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get t=txn4 k=k8 localUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan t=txn4 k=k8 localUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + + +run ok +txn_begin t=txn5 ts=5,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn5" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=15.000000000,0 + +run error +get t=txn5 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +get t=txn5 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + +run error +scan t=txn5 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=15.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn6 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn6" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run error +get t=txn6 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn6 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn6 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn7 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn7" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn7 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn7 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn7 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=10.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn8 ts=5,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn8" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=20.000000000,0 + +run error +get t=txn8 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +get t=txn8 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + +run error +scan t=txn8 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=20.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn9 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn9" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn9 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn9 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn9 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn10 ts=5,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn10" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn10 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn10 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn10 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn11 ts=10,0 globalUncertaintyLimit=15,0 +---- +>> at end: +txn: "txn11" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=15.000000000,0 + +run ok +get t=txn11 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn11 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn11 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn11 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn11 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn11 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn11 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn11 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn11 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn11 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn11 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn11 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn11 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn11 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn11 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn11 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn12 ts=10,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn12" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn12 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn12 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn12 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn12 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn12 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn12 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn12 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn12 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn12 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn12 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn12 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn12 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn12 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn12 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn12 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn12 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn13 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn13" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn13 k=k1 localUncertaintyLimit=10,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn13 k=k1 localUncertaintyLimit=10,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn13 k=k2 localUncertaintyLimit=10,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn13 k=k2 localUncertaintyLimit=10,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn13 k=k3 localUncertaintyLimit=10,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn13 k=k3 localUncertaintyLimit=10,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn13 k=k4 localUncertaintyLimit=10,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn13 k=k4 localUncertaintyLimit=10,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn13 k=k5 localUncertaintyLimit=10,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn13 k=k5 localUncertaintyLimit=10,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn13 k=k6 localUncertaintyLimit=10,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn13 k=k6 localUncertaintyLimit=10,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn13 k=k7 localUncertaintyLimit=10,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn13 k=k7 localUncertaintyLimit=10,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn13 k=k8 localUncertaintyLimit=10,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn13 k=k8 localUncertaintyLimit=10,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn14 ts=10,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn14" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn14 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn14 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn14 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn14 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn14 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn14 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn14 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn14 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn14 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn14 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn14 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn14 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn14 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn14 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn14 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn14 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn15 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn15" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn15 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn15 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn15 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn15 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn15 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn15 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn15 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn15 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn15 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn15 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn15 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn15 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn15 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn15 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn15 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn15 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn16 ts=10,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn16" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn16 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn16 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn16 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn17 ts=15,0 globalUncertaintyLimit=20,0 +---- +>> at end: +txn: "txn17" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=20.000000000,0 + +run ok +get t=txn17 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn17 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn17 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn17 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn17 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn17 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn17 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn17 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn17 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn17 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn17 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn17 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn17 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn17 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn17 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn17 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn18 ts=15,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn18" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn18 k=k1 localUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan t=txn18 k=k1 localUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get t=txn18 k=k2 localUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan t=txn18 k=k2 localUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get t=txn18 k=k3 localUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan t=txn18 k=k3 localUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get t=txn18 k=k4 localUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan t=txn18 k=k4 localUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get t=txn18 k=k5 localUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan t=txn18 k=k5 localUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get t=txn18 k=k6 localUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan t=txn18 k=k6 localUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get t=txn18 k=k7 localUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan t=txn18 k=k7 localUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get t=txn18 k=k8 localUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan t=txn18 k=k8 localUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + + +run ok +txn_begin t=txn19 ts=15,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn19" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false gul=25.000000000,0 + +run error +get t=txn19 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +get t=txn19 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + +run error +scan t=txn19 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=20.000000000,0, global=25.000000000,0)`; observed timestamps: [] + + +run ok +txn_begin t=txn20 ts=20,0 globalUncertaintyLimit=25,0 +---- +>> at end: +txn: "txn20" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false gul=25.000000000,0 + +run ok +get t=txn20 k=k1 localUncertaintyLimit=20,0 +---- +get: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +scan t=txn20 k=k1 localUncertaintyLimit=20,0 +---- +scan: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +get t=txn20 k=k2 localUncertaintyLimit=20,0 +---- +get: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +scan t=txn20 k=k2 localUncertaintyLimit=20,0 +---- +scan: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +get t=txn20 k=k3 localUncertaintyLimit=20,0 +---- +get: "k3" -> /BYTES/v6 @20.000000000,0 + +run ok +scan t=txn20 k=k3 localUncertaintyLimit=20,0 +---- +scan: "k3" -> /BYTES/v6 @20.000000000,0 + +run ok +get t=txn20 k=k4 localUncertaintyLimit=20,0 +---- +get: "k4" -> /BYTES/v8 @20.000000000,0 + +run ok +scan t=txn20 k=k4 localUncertaintyLimit=20,0 +---- +scan: "k4" -> /BYTES/v8 @20.000000000,0 + +run error +get t=txn20 k=k5 localUncertaintyLimit=20,0 +---- +get: "k5" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +scan t=txn20 k=k5 localUncertaintyLimit=20,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +get t=txn20 k=k6 localUncertaintyLimit=20,0 +---- +get: "k6" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +scan t=txn20 k=k6 localUncertaintyLimit=20,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +get t=txn20 k=k7 localUncertaintyLimit=20,0 +---- +get: "k7" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +scan t=txn20 k=k7 localUncertaintyLimit=20,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +get t=txn20 k=k8 localUncertaintyLimit=20,0 +---- +get: "k8" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +run error +scan t=txn20 k=k8 localUncertaintyLimit=20,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +# A subset of the previous test cases, but with non-transactional reads: +# +# for ts in (5, 15, 25): +# for localUncertaintyLimit in (5, 15, 25): +# if localUncertaintyLimit < ts: continue +# for globalUncertaintyLimit in (5, 15, 25): +# if globalUncertaintyLimit < ts: continue +# if globalUncertaintyLimit < localUncertaintyLimit: continue +# for k in (k1, k2, k3, k4, k5, k6, k7, k8): +# for op in (get, scan): +# testCase() +# + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k2" -> + +run ok +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k3" -> + +run ok +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k4" -> + +run ok +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k6" -> + +run ok +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k7" -> + +run ok +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +get: "k8" -> + +run ok +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=5,0 +---- +scan: "k8"-"k8\x00" -> + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> + +run ok +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> + +run ok +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> + +run ok +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> + +run ok +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> + +run ok +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> + +run ok +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> + +run ok +get k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> + +run ok +scan k=k1 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> + +run ok +get k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> + +run ok +scan k=k2 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> + +run ok +get k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> + +run ok +scan k=k3 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> + +run ok +get k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> + +run ok +scan k=k4 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> + +run ok +get k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> + +run ok +scan k=k5 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> + +run ok +get k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> + +run ok +scan k=k6 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> + +run ok +get k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> + +run ok +scan k=k7 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> + +run ok +get k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> + +run ok +scan k=k8 ts=5,0 localUncertaintyLimit=5,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> + +run error +get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= (local=15.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=5,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=15,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +get k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +scan k=k1 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k1" -> /BYTES/v1 @10.000000000,0 + +run ok +get k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +scan k=k2 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k2" -> /BYTES/v3 @10.000000000,0 + +run ok +get k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +scan k=k3 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k3" -> /BYTES/v5 @10.000000000,0 + +run ok +get k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +scan k=k4 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k4" -> /BYTES/v7 @10.000000000,0 + +run ok +get k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +scan k=k5 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k5" -> /BYTES/v9 @10.000000000,0 + +run ok +get k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +scan k=k6 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k6" -> /BYTES/v11 @10.000000000,0 + +run ok +get k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +scan k=k7 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k7" -> /BYTES/v13 @10.000000000,0 + +run ok +get k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> /BYTES/v15 @10.000000000,0 + +run ok +scan k=k8 ts=15,0 localUncertaintyLimit=15,0 globalUncertaintyLimit=25,0 +---- +scan: "k8" -> /BYTES/v15 @10.000000000,0 + +run error +get k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k1 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1"-"k1\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k2 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2"-"k2\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k3 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3"-"k3\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k4 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4"-"k4\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k5 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k6 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k7 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +get k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run error +scan k=k8 ts=15,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= (local=25.000000000,0, global=0,0)`; observed timestamps: [] + +run ok +get k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +scan k=k1 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k1" -> /BYTES/v2 @20.000000000,0 + +run ok +get k=k2 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +scan k=k2 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k2" -> /BYTES/v4 @20.000000000,0 + +run ok +get k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k3" -> /BYTES/v6 @20.000000000,0 + +run ok +scan k=k3 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k3" -> /BYTES/v6 @20.000000000,0 + +run ok +get k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k4" -> /BYTES/v8 @20.000000000,0 + +run ok +scan k=k4 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k4" -> /BYTES/v8 @20.000000000,0 + +run error +get k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k5" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +scan k=k5 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k5"-"k5\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k5" + +run error +get k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k6" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +scan k=k6 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k6"-"k6\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k6" + +run error +get k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k7" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +scan k=k7 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k7"-"k7\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k7" + +run error +get k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +get: "k8" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8" + +run error +scan k=k8 ts=25,0 localUncertaintyLimit=25,0 globalUncertaintyLimit=25,0 +---- +scan: "k8"-"k8\x00" -> +error: (*roachpb.WriteIntentError:) conflicting intents on "k8"