From d49e40bf597020f2aa502a2167f19859661e729b Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Thu, 15 Sep 2022 21:39:56 +0000 Subject: [PATCH 1/3] storage: remove usages of mvccEngineImpls The mvccEngineImpls variable and idiom throughout pkg/storage unit tests is a vestige from the migration from RocksDB to Pebble. Now that we have a single engine implementation once again, we should remove it and reclaim some tab characters. Release note: None --- pkg/storage/batch_test.go | 1130 ++-- pkg/storage/engine_test.go | 1544 +++-- pkg/storage/mvcc_incremental_iterator_test.go | 994 ++- pkg/storage/mvcc_logical_ops_test.go | 262 +- pkg/storage/mvcc_stats_test.go | 2273 ++++--- pkg/storage/mvcc_test.go | 5802 ++++++++--------- pkg/storage/sst_test.go | 72 +- 7 files changed, 5778 insertions(+), 6299 deletions(-) diff --git a/pkg/storage/batch_test.go b/pkg/storage/batch_test.go index 7a4362b03da6..96ce2b5f15be 100644 --- a/pkg/storage/batch_test.go +++ b/pkg/storage/batch_test.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -62,67 +61,63 @@ func appender(s string) []byte { } func testBatchBasics(t *testing.T, writeOnly bool, commit func(e Engine, b Batch) error) { - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() - - var b Batch - if writeOnly { - b = e.NewUnindexedBatch(true /* writeOnly */) - } else { - b = e.NewBatch() - } - defer b.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - require.NoError(t, b.PutUnversioned(mvccKey("a").Key, []byte("value"))) + var b Batch + if writeOnly { + b = e.NewUnindexedBatch(true /* writeOnly */) + } else { + b = e.NewBatch() + } + defer b.Close() - // Write an engine value to be deleted. - require.NoError(t, e.PutUnversioned(mvccKey("b").Key, []byte("value"))) - require.NoError(t, b.ClearUnversioned(mvccKey("b").Key)) + require.NoError(t, b.PutUnversioned(mvccKey("a").Key, []byte("value"))) - // Write an engine value to be merged. - require.NoError(t, e.PutUnversioned(mvccKey("c").Key, appender("foo"))) - require.NoError(t, b.Merge(mvccKey("c"), appender("bar"))) + // Write an engine value to be deleted. + require.NoError(t, e.PutUnversioned(mvccKey("b").Key, []byte("value"))) + require.NoError(t, b.ClearUnversioned(mvccKey("b").Key)) - // Write a key with an empty value. - require.NoError(t, b.PutUnversioned(mvccKey("e").Key, nil)) + // Write an engine value to be merged. + require.NoError(t, e.PutUnversioned(mvccKey("c").Key, appender("foo"))) + require.NoError(t, b.Merge(mvccKey("c"), appender("bar"))) - // Write an engine value to be single deleted. - require.NoError(t, e.PutUnversioned(mvccKey("d").Key, []byte("before"))) - require.NoError(t, b.SingleClearEngineKey(EngineKey{Key: mvccKey("d").Key})) + // Write a key with an empty value. + require.NoError(t, b.PutUnversioned(mvccKey("e").Key, nil)) - // Check all keys are in initial state (nothing from batch has gone - // through to engine until commit). - expValues := []MVCCKeyValue{ - {Key: mvccKey("b"), Value: []byte("value")}, - {Key: mvccKey("c"), Value: appender("foo")}, - {Key: mvccKey("d"), Value: []byte("before")}, - } - kvs, err := Scan(e, localMax, roachpb.KeyMax, 0) - require.NoError(t, err) - require.Equal(t, expValues, kvs) + // Write an engine value to be single deleted. + require.NoError(t, e.PutUnversioned(mvccKey("d").Key, []byte("before"))) + require.NoError(t, b.SingleClearEngineKey(EngineKey{Key: mvccKey("d").Key})) - // Now, merged values should be: - expValues = []MVCCKeyValue{ - {Key: mvccKey("a"), Value: []byte("value")}, - {Key: mvccKey("c"), Value: appender("foobar")}, - {Key: mvccKey("e"), Value: []byte{}}, - } - if !writeOnly { - // Scan values from batch directly. - kvs, err = Scan(b, localMax, roachpb.KeyMax, 0) - require.NoError(t, err) - require.Equal(t, expValues, kvs) - } + // Check all keys are in initial state (nothing from batch has gone + // through to engine until commit). + expValues := []MVCCKeyValue{ + {Key: mvccKey("b"), Value: []byte("value")}, + {Key: mvccKey("c"), Value: appender("foo")}, + {Key: mvccKey("d"), Value: []byte("before")}, + } + kvs, err := Scan(e, localMax, roachpb.KeyMax, 0) + require.NoError(t, err) + require.Equal(t, expValues, kvs) - // Commit batch and verify direct engine scan yields correct values. - require.NoError(t, commit(e, b)) - kvs, err = Scan(e, localMax, roachpb.KeyMax, 0) - require.NoError(t, err) - require.Equal(t, expValues, kvs) - }) + // Now, merged values should be: + expValues = []MVCCKeyValue{ + {Key: mvccKey("a"), Value: []byte("value")}, + {Key: mvccKey("c"), Value: appender("foobar")}, + {Key: mvccKey("e"), Value: []byte{}}, } + if !writeOnly { + // Scan values from batch directly. + kvs, err = Scan(b, localMax, roachpb.KeyMax, 0) + require.NoError(t, err) + require.Equal(t, expValues, kvs) + } + + // Commit batch and verify direct engine scan yields correct values. + require.NoError(t, commit(e, b)) + kvs, err = Scan(e, localMax, roachpb.KeyMax, 0) + require.NoError(t, err) + require.Equal(t, expValues, kvs) } // TestBatchBasics verifies that all commands work in a batch, aren't @@ -162,98 +157,90 @@ func TestReadOnlyBasics(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - ro := e.NewReadOnly(StandardDurability) - if ro.Closed() { - t.Fatal("read-only is expectedly found to be closed") - } - a := mvccKey("a") - successTestCases := []func(){ - func() { - _ = ro.MVCCIterate(a.Key, a.Key, MVCCKeyIterKind, IterKeyTypePointsOnly, - func(MVCCKeyValue, MVCCRangeKeyStack) error { return iterutil.StopIteration() }) - }, - func() { ro.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: roachpb.KeyMax}).Close() }, - func() { - ro.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ - MinTimestampHint: hlc.MinTimestamp, - MaxTimestampHint: hlc.MaxTimestamp, - UpperBound: roachpb.KeyMax, - }).Close() - }, - } - defer func(engineName string) { - ro.Close() - if !ro.Closed() { - t.Fatal("even after calling Close, a read-only should not be closed") - } - name := "rocksDBReadOnly" - if engineName == "pebble" { - name = "pebbleReadOnly" - } - shouldPanic(t, func() { ro.Close() }, "Close", "closing an already-closed "+name) - for i, f := range successTestCases { - shouldPanic(t, f, strconv.Itoa(i), "using a closed "+name) - } - }(engineImpl.name) + ro := e.NewReadOnly(StandardDurability) + if ro.Closed() { + t.Fatal("read-only is expectedly found to be closed") + } + a := mvccKey("a") + successTestCases := []func(){ + func() { + _ = ro.MVCCIterate(a.Key, a.Key, MVCCKeyIterKind, IterKeyTypePointsOnly, + func(MVCCKeyValue, MVCCRangeKeyStack) error { return iterutil.StopIteration() }) + }, + func() { ro.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: roachpb.KeyMax}).Close() }, + func() { + ro.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ + MinTimestampHint: hlc.MinTimestamp, + MaxTimestampHint: hlc.MaxTimestamp, + UpperBound: roachpb.KeyMax, + }).Close() + }, + } + defer func() { + ro.Close() + if !ro.Closed() { + t.Fatal("even after calling Close, a read-only should not be closed") + } + shouldPanic(t, func() { ro.Close() }, "Close", "closing an already-closed pebbleReadOnly") + for i, f := range successTestCases { + shouldPanic(t, f, strconv.Itoa(i), "using a closed pebbleReadOnly") + } + }() - for i, f := range successTestCases { - shouldNotPanic(t, f, strconv.Itoa(i)) - } + for i, f := range successTestCases { + shouldNotPanic(t, f, strconv.Itoa(i)) + } - // For a read-only ReadWriter, all Writer methods should panic. - failureTestCases := []func(){ - func() { _ = ro.ApplyBatchRepr(nil, false) }, - func() { _ = ro.ClearUnversioned(a.Key) }, - func() { _ = ro.SingleClearEngineKey(EngineKey{Key: a.Key}) }, - func() { _ = ro.ClearRawRange(a.Key, a.Key, true, true) }, - func() { _ = ro.Merge(a, nil) }, - func() { _ = ro.PutUnversioned(a.Key, nil) }, - } - for i, f := range failureTestCases { - shouldPanic(t, f, strconv.Itoa(i), "not implemented") - } + // For a read-only ReadWriter, all Writer methods should panic. + failureTestCases := []func(){ + func() { _ = ro.ApplyBatchRepr(nil, false) }, + func() { _ = ro.ClearUnversioned(a.Key) }, + func() { _ = ro.SingleClearEngineKey(EngineKey{Key: a.Key}) }, + func() { _ = ro.ClearRawRange(a.Key, a.Key, true, true) }, + func() { _ = ro.Merge(a, nil) }, + func() { _ = ro.PutUnversioned(a.Key, nil) }, + } + for i, f := range failureTestCases { + shouldPanic(t, f, strconv.Itoa(i), "not implemented") + } - if err := e.PutUnversioned(mvccKey("a").Key, []byte("value")); err != nil { - t.Fatal(err) - } - if err := e.PutUnversioned(mvccKey("b").Key, []byte("value")); err != nil { - t.Fatal(err) - } - if err := e.ClearUnversioned(mvccKey("b").Key); err != nil { - t.Fatal(err) - } - if err := e.PutUnversioned(mvccKey("c").Key, appender("foo")); err != nil { - t.Fatal(err) - } - if err := e.Merge(mvccKey("c"), appender("bar")); err != nil { - t.Fatal(err) - } - if err := e.PutUnversioned(mvccKey("d").Key, []byte("value")); err != nil { - t.Fatal(err) - } - if err := e.SingleClearEngineKey(EngineKey{Key: mvccKey("d").Key}); err != nil { - t.Fatal(err) - } + if err := e.PutUnversioned(mvccKey("a").Key, []byte("value")); err != nil { + t.Fatal(err) + } + if err := e.PutUnversioned(mvccKey("b").Key, []byte("value")); err != nil { + t.Fatal(err) + } + if err := e.ClearUnversioned(mvccKey("b").Key); err != nil { + t.Fatal(err) + } + if err := e.PutUnversioned(mvccKey("c").Key, appender("foo")); err != nil { + t.Fatal(err) + } + if err := e.Merge(mvccKey("c"), appender("bar")); err != nil { + t.Fatal(err) + } + if err := e.PutUnversioned(mvccKey("d").Key, []byte("value")); err != nil { + t.Fatal(err) + } + if err := e.SingleClearEngineKey(EngineKey{Key: mvccKey("d").Key}); err != nil { + t.Fatal(err) + } - // Now, merged values should be: - expValues := []MVCCKeyValue{ - {Key: mvccKey("a"), Value: []byte("value")}, - {Key: mvccKey("c"), Value: appender("foobar")}, - } + // Now, merged values should be: + expValues := []MVCCKeyValue{ + {Key: mvccKey("a"), Value: []byte("value")}, + {Key: mvccKey("c"), Value: appender("foobar")}, + } - kvs, err := Scan(e, localMax, roachpb.KeyMax, 0) - if err != nil { - t.Fatal(err) - } - if !reflect.DeepEqual(expValues, kvs) { - t.Errorf("%v != %v", kvs, expValues) - } - }) + kvs, err := Scan(e, localMax, roachpb.KeyMax, 0) + if err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(expValues, kvs) { + t.Errorf("%v != %v", kvs, expValues) } } @@ -319,61 +306,57 @@ func TestApplyBatchRepr(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - // Failure to represent the absorbed Batch again. - { - b1 := e.NewBatch() - defer b1.Close() + // Failure to represent the absorbed Batch again. + { + b1 := e.NewBatch() + defer b1.Close() - if err := b1.PutUnversioned(mvccKey("lost").Key, []byte("update")); err != nil { - t.Fatal(err) - } + if err := b1.PutUnversioned(mvccKey("lost").Key, []byte("update")); err != nil { + t.Fatal(err) + } - repr1 := b1.Repr() + repr1 := b1.Repr() - b2 := e.NewBatch() - defer b2.Close() - if err := b2.ApplyBatchRepr(repr1, false /* sync */); err != nil { - t.Fatal(err) - } - repr2 := b2.Repr() + b2 := e.NewBatch() + defer b2.Close() + if err := b2.ApplyBatchRepr(repr1, false /* sync */); err != nil { + t.Fatal(err) + } + repr2 := b2.Repr() - if !reflect.DeepEqual(repr1, repr2) { - t.Fatalf("old batch represents to:\n%q\nrestored batch to:\n%q", repr1, repr2) - } - } + if !reflect.DeepEqual(repr1, repr2) { + t.Fatalf("old batch represents to:\n%q\nrestored batch to:\n%q", repr1, repr2) + } + } - // Failure to commit what was absorbed. - { - b3 := e.NewBatch() - defer b3.Close() + // Failure to commit what was absorbed. + { + b3 := e.NewBatch() + defer b3.Close() - key := mvccKey("phantom") - val := []byte("phantom") + key := mvccKey("phantom") + val := []byte("phantom") - if err := b3.PutUnversioned(key.Key, val); err != nil { - t.Fatal(err) - } + if err := b3.PutUnversioned(key.Key, val); err != nil { + t.Fatal(err) + } - repr := b3.Repr() + repr := b3.Repr() - b4 := e.NewBatch() - defer b4.Close() - if err := b4.ApplyBatchRepr(repr, false /* sync */); err != nil { - t.Fatal(err) - } - // Intentionally don't call Repr() because the expected user wouldn't. - if err := b4.Commit(false /* sync */); err != nil { - t.Fatal(err) - } + b4 := e.NewBatch() + defer b4.Close() + if err := b4.ApplyBatchRepr(repr, false /* sync */); err != nil { + t.Fatal(err) + } + // Intentionally don't call Repr() because the expected user wouldn't. + if err := b4.Commit(false /* sync */); err != nil { + t.Fatal(err) + } - require.Equal(t, val, mvccGetRaw(t, e, key)) - } - }) + require.Equal(t, val, mvccGetRaw(t, e, key)) } } @@ -381,51 +364,47 @@ func TestBatchGet(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - b := e.NewBatch() - defer b.Close() + b := e.NewBatch() + defer b.Close() - // Write initial values, then write to batch. - if err := e.PutUnversioned(mvccKey("b").Key, []byte("value")); err != nil { - t.Fatal(err) - } - if err := e.PutUnversioned(mvccKey("c").Key, appender("foo")); err != nil { - t.Fatal(err) - } - // Write batch values. - if err := b.PutUnversioned(mvccKey("a").Key, []byte("value")); err != nil { - t.Fatal(err) - } - if err := b.ClearUnversioned(mvccKey("b").Key); err != nil { - t.Fatal(err) - } - if err := b.Merge(mvccKey("c"), appender("bar")); err != nil { - t.Fatal(err) - } - if err := b.PutUnversioned(mvccKey("d").Key, []byte("before")); err != nil { - t.Fatal(err) - } - if err := b.SingleClearEngineKey(EngineKey{Key: mvccKey("d").Key}); err != nil { - t.Fatal(err) - } - if err := b.PutUnversioned(mvccKey("d").Key, []byte("after")); err != nil { - t.Fatal(err) - } + // Write initial values, then write to batch. + if err := e.PutUnversioned(mvccKey("b").Key, []byte("value")); err != nil { + t.Fatal(err) + } + if err := e.PutUnversioned(mvccKey("c").Key, appender("foo")); err != nil { + t.Fatal(err) + } + // Write batch values. + if err := b.PutUnversioned(mvccKey("a").Key, []byte("value")); err != nil { + t.Fatal(err) + } + if err := b.ClearUnversioned(mvccKey("b").Key); err != nil { + t.Fatal(err) + } + if err := b.Merge(mvccKey("c"), appender("bar")); err != nil { + t.Fatal(err) + } + if err := b.PutUnversioned(mvccKey("d").Key, []byte("before")); err != nil { + t.Fatal(err) + } + if err := b.SingleClearEngineKey(EngineKey{Key: mvccKey("d").Key}); err != nil { + t.Fatal(err) + } + if err := b.PutUnversioned(mvccKey("d").Key, []byte("after")); err != nil { + t.Fatal(err) + } - expValues := []MVCCKeyValue{ - {Key: mvccKey("a"), Value: []byte("value")}, - {Key: mvccKey("b"), Value: nil}, - {Key: mvccKey("c"), Value: appender("foobar")}, - {Key: mvccKey("d"), Value: []byte("after")}, - } - for _, expKV := range expValues { - require.Equal(t, expKV.Value, mvccGetRaw(t, b, expKV.Key)) - } - }) + expValues := []MVCCKeyValue{ + {Key: mvccKey("a"), Value: []byte("value")}, + {Key: mvccKey("b"), Value: nil}, + {Key: mvccKey("c"), Value: appender("foobar")}, + {Key: mvccKey("d"), Value: []byte("after")}, + } + for _, expKV := range expValues { + require.Equal(t, expKV.Value, mvccGetRaw(t, b, expKV.Key)) } } @@ -444,52 +423,48 @@ func TestBatchMerge(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - b := e.NewBatch() - defer b.Close() + b := e.NewBatch() + defer b.Close() - // Write batch put, delete & merge. - if err := b.PutUnversioned(mvccKey("a").Key, appender("a-value")); err != nil { - t.Fatal(err) - } - if err := b.ClearUnversioned(mvccKey("b").Key); err != nil { - t.Fatal(err) - } - if err := b.Merge(mvccKey("c"), appender("c-value")); err != nil { - t.Fatal(err) - } + // Write batch put, delete & merge. + if err := b.PutUnversioned(mvccKey("a").Key, appender("a-value")); err != nil { + t.Fatal(err) + } + if err := b.ClearUnversioned(mvccKey("b").Key); err != nil { + t.Fatal(err) + } + if err := b.Merge(mvccKey("c"), appender("c-value")); err != nil { + t.Fatal(err) + } - // Now, merge to all three keys. - if err := b.Merge(mvccKey("a"), appender("append")); err != nil { - t.Fatal(err) - } - if err := b.Merge(mvccKey("b"), appender("append")); err != nil { - t.Fatal(err) - } - if err := b.Merge(mvccKey("c"), appender("append")); err != nil { - t.Fatal(err) - } + // Now, merge to all three keys. + if err := b.Merge(mvccKey("a"), appender("append")); err != nil { + t.Fatal(err) + } + if err := b.Merge(mvccKey("b"), appender("append")); err != nil { + t.Fatal(err) + } + if err := b.Merge(mvccKey("c"), appender("append")); err != nil { + t.Fatal(err) + } - // Verify values. - val := mvccGetRaw(t, b, mvccKey("a")) - if !compareMergedValues(t, val, appender("a-valueappend")) { - t.Error("mismatch of \"a\"") - } + // Verify values. + val := mvccGetRaw(t, b, mvccKey("a")) + if !compareMergedValues(t, val, appender("a-valueappend")) { + t.Error("mismatch of \"a\"") + } - val = mvccGetRaw(t, b, mvccKey("b")) - if !compareMergedValues(t, val, appender("append")) { - t.Error("mismatch of \"b\"") - } + val = mvccGetRaw(t, b, mvccKey("b")) + if !compareMergedValues(t, val, appender("append")) { + t.Error("mismatch of \"b\"") + } - val = mvccGetRaw(t, b, mvccKey("c")) - if !compareMergedValues(t, val, appender("c-valueappend")) { - t.Error("mismatch of \"c\"") - } - }) + val = mvccGetRaw(t, b, mvccKey("c")) + if !compareMergedValues(t, val, appender("c-valueappend")) { + t.Error("mismatch of \"c\"") } } @@ -497,95 +472,91 @@ func TestBatchScan(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - b := e.NewBatch() - defer b.Close() + b := e.NewBatch() + defer b.Close() - existingVals := []MVCCKeyValue{ - {Key: mvccKey("a"), Value: []byte("1")}, - {Key: mvccKey("b"), Value: []byte("2")}, - {Key: mvccKey("c"), Value: []byte("3")}, - {Key: mvccKey("d"), Value: []byte("4")}, - {Key: mvccKey("e"), Value: []byte("5")}, - {Key: mvccKey("f"), Value: []byte("6")}, - {Key: mvccKey("g"), Value: []byte("7")}, - {Key: mvccKey("h"), Value: []byte("8")}, - {Key: mvccKey("i"), Value: []byte("9")}, - {Key: mvccKey("j"), Value: []byte("10")}, - {Key: mvccKey("k"), Value: []byte("11")}, - {Key: mvccKey("l"), Value: []byte("12")}, - {Key: mvccKey("m"), Value: []byte("13")}, - } - for _, kv := range existingVals { - if err := e.PutUnversioned(kv.Key.Key, kv.Value); err != nil { - t.Fatal(err) - } - } + existingVals := []MVCCKeyValue{ + {Key: mvccKey("a"), Value: []byte("1")}, + {Key: mvccKey("b"), Value: []byte("2")}, + {Key: mvccKey("c"), Value: []byte("3")}, + {Key: mvccKey("d"), Value: []byte("4")}, + {Key: mvccKey("e"), Value: []byte("5")}, + {Key: mvccKey("f"), Value: []byte("6")}, + {Key: mvccKey("g"), Value: []byte("7")}, + {Key: mvccKey("h"), Value: []byte("8")}, + {Key: mvccKey("i"), Value: []byte("9")}, + {Key: mvccKey("j"), Value: []byte("10")}, + {Key: mvccKey("k"), Value: []byte("11")}, + {Key: mvccKey("l"), Value: []byte("12")}, + {Key: mvccKey("m"), Value: []byte("13")}, + } + for _, kv := range existingVals { + if err := e.PutUnversioned(kv.Key.Key, kv.Value); err != nil { + t.Fatal(err) + } + } - batchVals := []MVCCKeyValue{ - {Key: mvccKey("a"), Value: []byte("b1")}, - {Key: mvccKey("bb"), Value: []byte("b2")}, - {Key: mvccKey("c"), Value: []byte("b3")}, - {Key: mvccKey("dd"), Value: []byte("b4")}, - {Key: mvccKey("e"), Value: []byte("b5")}, - {Key: mvccKey("ff"), Value: []byte("b6")}, - {Key: mvccKey("g"), Value: []byte("b7")}, - {Key: mvccKey("hh"), Value: []byte("b8")}, - {Key: mvccKey("i"), Value: []byte("b9")}, - {Key: mvccKey("jj"), Value: []byte("b10")}, - } - for _, kv := range batchVals { - if err := b.PutUnversioned(kv.Key.Key, kv.Value); err != nil { - t.Fatal(err) - } - } + batchVals := []MVCCKeyValue{ + {Key: mvccKey("a"), Value: []byte("b1")}, + {Key: mvccKey("bb"), Value: []byte("b2")}, + {Key: mvccKey("c"), Value: []byte("b3")}, + {Key: mvccKey("dd"), Value: []byte("b4")}, + {Key: mvccKey("e"), Value: []byte("b5")}, + {Key: mvccKey("ff"), Value: []byte("b6")}, + {Key: mvccKey("g"), Value: []byte("b7")}, + {Key: mvccKey("hh"), Value: []byte("b8")}, + {Key: mvccKey("i"), Value: []byte("b9")}, + {Key: mvccKey("jj"), Value: []byte("b10")}, + } + for _, kv := range batchVals { + if err := b.PutUnversioned(kv.Key.Key, kv.Value); err != nil { + t.Fatal(err) + } + } - scans := []struct { - start, end roachpb.Key - max int64 - }{ - // Full monty. - {start: roachpb.Key("a"), end: roachpb.Key("z"), max: 0}, - // Select ~half. - {start: roachpb.Key("a"), end: roachpb.Key("z"), max: 9}, - // Select one. - {start: roachpb.Key("a"), end: roachpb.Key("z"), max: 1}, - // Select half by end key. - {start: roachpb.Key("a"), end: roachpb.Key("f0"), max: 0}, - // Start at half and select rest. - {start: roachpb.Key("f"), end: roachpb.Key("z"), max: 0}, - // Start at last and select max=10. - {start: roachpb.Key("m"), end: roachpb.Key("z"), max: 10}, - } + scans := []struct { + start, end roachpb.Key + max int64 + }{ + // Full monty. + {start: roachpb.Key("a"), end: roachpb.Key("z"), max: 0}, + // Select ~half. + {start: roachpb.Key("a"), end: roachpb.Key("z"), max: 9}, + // Select one. + {start: roachpb.Key("a"), end: roachpb.Key("z"), max: 1}, + // Select half by end key. + {start: roachpb.Key("a"), end: roachpb.Key("f0"), max: 0}, + // Start at half and select rest. + {start: roachpb.Key("f"), end: roachpb.Key("z"), max: 0}, + // Start at last and select max=10. + {start: roachpb.Key("m"), end: roachpb.Key("z"), max: 10}, + } - // Scan each case using the batch and store the results. - results := map[int][]MVCCKeyValue{} - for i, scan := range scans { - kvs, err := Scan(b, scan.start, scan.end, scan.max) - if err != nil { - t.Fatal(err) - } - results[i] = kvs - } + // Scan each case using the batch and store the results. + results := map[int][]MVCCKeyValue{} + for i, scan := range scans { + kvs, err := Scan(b, scan.start, scan.end, scan.max) + if err != nil { + t.Fatal(err) + } + results[i] = kvs + } - // Now, commit batch and re-scan using engine direct to compare results. - if err := b.Commit(false /* sync */); err != nil { - t.Fatal(err) - } - for i, scan := range scans { - kvs, err := Scan(e, scan.start, scan.end, scan.max) - if err != nil { - t.Fatal(err) - } - if !reflect.DeepEqual(kvs, results[i]) { - t.Errorf("%d: expected %v; got %v", i, results[i], kvs) - } - } - }) + // Now, commit batch and re-scan using engine direct to compare results. + if err := b.Commit(false /* sync */); err != nil { + t.Fatal(err) + } + for i, scan := range scans { + kvs, err := Scan(e, scan.start, scan.end, scan.max) + if err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(kvs, results[i]) { + t.Errorf("%d: expected %v; got %v", i, results[i], kvs) + } } } @@ -595,29 +566,25 @@ func TestBatchScanWithDelete(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - b := e.NewBatch() - defer b.Close() + b := e.NewBatch() + defer b.Close() - // Write initial value, then delete via batch. - if err := e.PutUnversioned(mvccKey("a").Key, []byte("value")); err != nil { - t.Fatal(err) - } - if err := b.ClearUnversioned(mvccKey("a").Key); err != nil { - t.Fatal(err) - } - kvs, err := Scan(b, localMax, roachpb.KeyMax, 0) - if err != nil { - t.Fatal(err) - } - if len(kvs) != 0 { - t.Errorf("expected empty scan with batch-deleted value; got %v", kvs) - } - }) + // Write initial value, then delete via batch. + if err := e.PutUnversioned(mvccKey("a").Key, []byte("value")); err != nil { + t.Fatal(err) + } + if err := b.ClearUnversioned(mvccKey("a").Key); err != nil { + t.Fatal(err) + } + kvs, err := Scan(b, localMax, roachpb.KeyMax, 0) + if err != nil { + t.Fatal(err) + } + if len(kvs) != 0 { + t.Errorf("expected empty scan with batch-deleted value; got %v", kvs) } } @@ -628,34 +595,30 @@ func TestBatchScanMaxWithDeleted(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - b := e.NewBatch() - defer b.Close() + b := e.NewBatch() + defer b.Close() - // Write two values. - if err := e.PutUnversioned(mvccKey("a").Key, []byte("value1")); err != nil { - t.Fatal(err) - } - if err := e.PutUnversioned(mvccKey("b").Key, []byte("value2")); err != nil { - t.Fatal(err) - } - // Now, delete "a" in batch. - if err := b.ClearUnversioned(mvccKey("a").Key); err != nil { - t.Fatal(err) - } - // A scan with max=1 should scan "b". - kvs, err := Scan(b, localMax, roachpb.KeyMax, 1) - if err != nil { - t.Fatal(err) - } - if len(kvs) != 1 || !bytes.Equal(kvs[0].Key.Key, []byte("b")) { - t.Errorf("expected scan of \"b\"; got %v", kvs) - } - }) + // Write two values. + if err := e.PutUnversioned(mvccKey("a").Key, []byte("value1")); err != nil { + t.Fatal(err) + } + if err := e.PutUnversioned(mvccKey("b").Key, []byte("value2")); err != nil { + t.Fatal(err) + } + // Now, delete "a" in batch. + if err := b.ClearUnversioned(mvccKey("a").Key); err != nil { + t.Fatal(err) + } + // A scan with max=1 should scan "b". + kvs, err := Scan(b, localMax, roachpb.KeyMax, 1) + if err != nil { + t.Fatal(err) + } + if len(kvs) != 1 || !bytes.Equal(kvs[0].Key.Key, []byte("b")) { + t.Errorf("expected scan of \"b\"; got %v", kvs) } } @@ -663,103 +626,91 @@ func TestBatchVisibleAfterApplyBatchRepr(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - wb := func() []byte { - batch := e.NewBatch() - defer batch.Close() + wb := func() []byte { + batch := e.NewBatch() + defer batch.Close() - if err := batch.PutUnversioned(mvccKey("batchkey").Key, []byte("b")); err != nil { - t.Fatal(err) - } + if err := batch.PutUnversioned(mvccKey("batchkey").Key, []byte("b")); err != nil { + t.Fatal(err) + } - return batch.Repr() - }() + return batch.Repr() + }() - batch := e.NewBatch() - defer batch.Close() + batch := e.NewBatch() + defer batch.Close() - assert.NoError(t, batch.ApplyBatchRepr(wb, false /* sync */)) + assert.NoError(t, batch.ApplyBatchRepr(wb, false /* sync */)) - // The batch can see the earlier write. - require.Equal(t, []byte("b"), mvccGetRaw(t, batch, mvccKey("batchkey"))) - }) - } + // The batch can see the earlier write. + require.Equal(t, []byte("b"), mvccGetRaw(t, batch, mvccKey("batchkey"))) } func TestUnindexedBatchThatSupportsReader(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() - - if err := e.PutUnversioned(mvccKey("b").Key, []byte("b")); err != nil { - t.Fatal(err) - } + e := NewDefaultInMemForTesting() + defer e.Close() - b := e.NewUnindexedBatch(false /* writeOnly */) - defer b.Close() - if err := b.PutUnversioned(mvccKey("b").Key, []byte("c")); err != nil { - t.Fatal(err) - } + if err := e.PutUnversioned(mvccKey("b").Key, []byte("b")); err != nil { + t.Fatal(err) + } - // Verify that reads on the distinct batch go to the underlying engine, not - // to the unindexed batch. - iter := b.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: roachpb.KeyMax}) - iter.SeekGE(mvccKey("a")) - if ok, err := iter.Valid(); !ok { - t.Fatalf("expected iterator to be valid, err=%v", err) - } - if string(iter.Key().Key) != "b" { - t.Fatalf("expected b, but got %s", iter.Key()) - } - iter.Close() + b := e.NewUnindexedBatch(false /* writeOnly */) + defer b.Close() + if err := b.PutUnversioned(mvccKey("b").Key, []byte("c")); err != nil { + t.Fatal(err) + } - require.Equal(t, []byte("b"), mvccGetRaw(t, b, mvccKey("b"))) - require.NoError(t, b.Commit(true)) - require.Equal(t, []byte("c"), mvccGetRaw(t, e, mvccKey("b"))) - }) + // Verify that reads on the distinct batch go to the underlying engine, not + // to the unindexed batch. + iter := b.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: roachpb.KeyMax}) + iter.SeekGE(mvccKey("a")) + if ok, err := iter.Valid(); !ok { + t.Fatalf("expected iterator to be valid, err=%v", err) + } + if string(iter.Key().Key) != "b" { + t.Fatalf("expected b, but got %s", iter.Key()) } + iter.Close() + + require.Equal(t, []byte("b"), mvccGetRaw(t, b, mvccKey("b"))) + require.NoError(t, b.Commit(true)) + require.Equal(t, []byte("c"), mvccGetRaw(t, e, mvccKey("b"))) } func TestUnindexedBatchThatDoesNotSupportReaderPanics(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - batch := e.NewUnindexedBatch(true /* writeOnly */) - defer batch.Close() + batch := e.NewUnindexedBatch(true /* writeOnly */) + defer batch.Close() - // The various Reader methods on the batch should panic. - a := mvccKey("a") - b := mvccKey("b") - testCases := []func(){ - func() { _ = batch.MVCCIterate(a.Key, b.Key, MVCCKeyIterKind, IterKeyTypePointsOnly, nil) }, - func() { _ = batch.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: roachpb.KeyMax}) }, - } - for i, f := range testCases { - func() { - defer func(i int) { - if r := recover(); r == nil { - t.Fatalf("%d: test did not panic", i) - } else if r != "write-only batch" { - t.Fatalf("%d: unexpected panic: %v", i, r) - } - }(i) - f() - }() - } - }) + // The various Reader methods on the batch should panic. + a := mvccKey("a") + b := mvccKey("b") + testCases := []func(){ + func() { _ = batch.MVCCIterate(a.Key, b.Key, MVCCKeyIterKind, IterKeyTypePointsOnly, nil) }, + func() { _ = batch.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: roachpb.KeyMax}) }, + } + for i, f := range testCases { + func() { + defer func(i int) { + if r := recover(); r == nil { + t.Fatalf("%d: test did not panic", i) + } else if r != "write-only batch" { + t.Fatalf("%d: unexpected panic: %v", i, r) + } + }(i) + f() + }() } } @@ -767,104 +718,81 @@ func TestBatchIteration(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() - - b := e.NewBatch() - defer b.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - k1 := MakeMVCCMetadataKey(roachpb.Key("c")) - k2 := MakeMVCCMetadataKey(roachpb.Key("d")) - k3 := MakeMVCCMetadataKey(roachpb.Key("e")) - v1 := []byte("value1") - v2 := []byte("value2") + b := e.NewBatch() + defer b.Close() - if err := b.PutUnversioned(k1.Key, v1); err != nil { - t.Fatal(err) - } - if err := b.PutUnversioned(k2.Key, v2); err != nil { - t.Fatal(err) - } - if err := b.PutUnversioned(k3.Key, []byte("doesn't matter")); err != nil { - t.Fatal(err) - } + k1 := MakeMVCCMetadataKey(roachpb.Key("c")) + k2 := MakeMVCCMetadataKey(roachpb.Key("d")) + k3 := MakeMVCCMetadataKey(roachpb.Key("e")) + v1 := []byte("value1") + v2 := []byte("value2") - iterOpts := IterOptions{UpperBound: k3.Key} - iter := b.NewMVCCIterator(MVCCKeyIterKind, iterOpts) - defer iter.Close() + if err := b.PutUnversioned(k1.Key, v1); err != nil { + t.Fatal(err) + } + if err := b.PutUnversioned(k2.Key, v2); err != nil { + t.Fatal(err) + } + if err := b.PutUnversioned(k3.Key, []byte("doesn't matter")); err != nil { + t.Fatal(err) + } - // Forward iteration, - iter.SeekGE(k1) - if ok, err := iter.Valid(); !ok { - t.Fatal(err) - } - if !reflect.DeepEqual(iter.Key(), k1) { - t.Fatalf("expected %s, got %s", k1, iter.Key()) - } - if !reflect.DeepEqual(iter.Value(), v1) { - t.Fatalf("expected %s, got %s", v1, iter.Value()) - } - iter.Next() - if ok, err := iter.Valid(); !ok { - t.Fatal(err) - } - if !reflect.DeepEqual(iter.Key(), k2) { - t.Fatalf("expected %s, got %s", k2, iter.Key()) - } - if !reflect.DeepEqual(iter.Value(), v2) { - t.Fatalf("expected %s, got %s", v2, iter.Value()) - } - iter.Next() - if ok, err := iter.Valid(); err != nil { - t.Fatal(err) - } else if ok { - t.Fatalf("expected invalid, got valid at key %s", iter.Key()) - } + iterOpts := IterOptions{UpperBound: k3.Key} + iter := b.NewMVCCIterator(MVCCKeyIterKind, iterOpts) + defer iter.Close() - // Reverse iteration. - switch engineImpl.name { - case "pebble": - // Reverse iteration in batches works on Pebble. - iter.SeekLT(k3) - if ok, err := iter.Valid(); !ok { - t.Fatal(err) - } - if !reflect.DeepEqual(iter.Key(), k2) { - t.Fatalf("expected %s, got %s", k2, iter.Key()) - } - if !reflect.DeepEqual(iter.Value(), v2) { - t.Fatalf("expected %s, got %s", v2, iter.Value()) - } + // Forward iteration, + iter.SeekGE(k1) + if ok, err := iter.Valid(); !ok { + t.Fatal(err) + } + if !reflect.DeepEqual(iter.Key(), k1) { + t.Fatalf("expected %s, got %s", k1, iter.Key()) + } + if !reflect.DeepEqual(iter.Value(), v1) { + t.Fatalf("expected %s, got %s", v1, iter.Value()) + } + iter.Next() + if ok, err := iter.Valid(); !ok { + t.Fatal(err) + } + if !reflect.DeepEqual(iter.Key(), k2) { + t.Fatalf("expected %s, got %s", k2, iter.Key()) + } + if !reflect.DeepEqual(iter.Value(), v2) { + t.Fatalf("expected %s, got %s", v2, iter.Value()) + } + iter.Next() + if ok, err := iter.Valid(); err != nil { + t.Fatal(err) + } else if ok { + t.Fatalf("expected invalid, got valid at key %s", iter.Key()) + } - iter.Prev() - if ok, err := iter.Valid(); !ok || err != nil { - t.Fatalf("expected success, but got invalid: %v", err) - } - if !reflect.DeepEqual(iter.Key(), k1) { - t.Fatalf("expected %s, got %s", k1, iter.Key()) - } - if !reflect.DeepEqual(iter.Value(), v1) { - t.Fatalf("expected %s, got %s", v1, iter.Value()) - } - default: - // Reverse iteration in batches is not supported with RocksDB. - iter.SeekLT(k3) - if ok, err := iter.Valid(); ok { - t.Fatalf("expected invalid, got valid at key %s", iter.Key()) - } else if !testutils.IsError(err, "SeekForPrev\\(\\) not supported") { - t.Fatalf("expected 'SeekForPrev() not supported', got %s", err) - } + // Reverse iteration. + iter.SeekLT(k3) + if ok, err := iter.Valid(); !ok { + t.Fatal(err) + } + if !reflect.DeepEqual(iter.Key(), k2) { + t.Fatalf("expected %s, got %s", k2, iter.Key()) + } + if !reflect.DeepEqual(iter.Value(), v2) { + t.Fatalf("expected %s, got %s", v2, iter.Value()) + } - iter.Prev() - if ok, err := iter.Valid(); ok { - t.Fatalf("expected invalid, got valid at key %s", iter.Key()) - } else if !testutils.IsError(err, "Prev\\(\\) not supported") { - t.Fatalf("expected 'Prev() not supported', got %s", err) - } - } - }) + iter.Prev() + if ok, err := iter.Valid(); !ok || err != nil { + t.Fatalf("expected success, but got invalid: %v", err) + } + if !reflect.DeepEqual(iter.Key(), k1) { + t.Fatalf("expected %s, got %s", k1, iter.Key()) + } + if !reflect.DeepEqual(iter.Value(), v1) { + t.Fatalf("expected %s, got %s", v1, iter.Value()) } } @@ -874,53 +802,49 @@ func TestBatchCombine(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() - - var n uint32 - const count = 10000 - - errs := make(chan error, 10) - for i := 0; i < cap(errs); i++ { - go func() { - for { - v := atomic.AddUint32(&n, 1) - 1 - if v >= count { - break - } - k := fmt.Sprint(v) - - b := e.NewUnindexedBatch(true /* writeOnly */) - if err := b.PutUnversioned(mvccKey(k).Key, []byte(k)); err != nil { - errs <- errors.Wrap(err, "put failed") - return - } - if err := b.Commit(false); err != nil { - errs <- errors.Wrap(err, "commit failed") - return - } - - // Verify we can read the key we just wrote immediately. - if v, err := mvccGetRawWithError(t, e, mvccKey(k)); err != nil { - errs <- errors.Wrap(err, "get failed") - return - } else if string(v) != k { - errs <- errors.Errorf("read %q from engine, expected %q", v, k) - return - } - } - errs <- nil - }() - } + e := NewDefaultInMemForTesting() + defer e.Close() + + var n uint32 + const count = 10000 - for i := 0; i < cap(errs); i++ { - if err := <-errs; err != nil { - t.Error(err) + errs := make(chan error, 10) + for i := 0; i < cap(errs); i++ { + go func() { + for { + v := atomic.AddUint32(&n, 1) - 1 + if v >= count { + break + } + k := fmt.Sprint(v) + + b := e.NewUnindexedBatch(true /* writeOnly */) + if err := b.PutUnversioned(mvccKey(k).Key, []byte(k)); err != nil { + errs <- errors.Wrap(err, "put failed") + return + } + if err := b.Commit(false); err != nil { + errs <- errors.Wrap(err, "commit failed") + return + } + + // Verify we can read the key we just wrote immediately. + if v, err := mvccGetRawWithError(t, e, mvccKey(k)); err != nil { + errs <- errors.Wrap(err, "get failed") + return + } else if string(v) != k { + errs <- errors.Errorf("read %q from engine, expected %q", v, k) + return } } - }) + errs <- nil + }() + } + + for i := 0; i < cap(errs); i++ { + if err := <-errs; err != nil { + t.Error(err) + } } } diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 0821d4096914..51144d208042 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -14,7 +14,6 @@ import ( "bytes" "context" "encoding/binary" - "encoding/hex" "fmt" "io" "math" @@ -75,57 +74,53 @@ func TestEngineBatchCommit(t *testing.T) { key := mvccKey("a") finalVal := []byte(strconv.Itoa(numWrites - 1)) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() - - // Start a concurrent read operation in a busy loop. - readsBegun := make(chan struct{}) - readsDone := make(chan error) - writesDone := make(chan struct{}) - go func() { - readsDone <- func() error { - readsBegunAlias := readsBegun - for { - select { - case <-writesDone: - return nil - default: - val, err := mvccGetRawWithError(t, e, key) - if err != nil { - return err - } - if val != nil && !bytes.Equal(val, finalVal) { - return errors.Errorf("key value should be empty or %q; got %q", string(finalVal), string(val)) - } - if readsBegunAlias != nil { - close(readsBegunAlias) - readsBegunAlias = nil - } - } + e := NewDefaultInMemForTesting() + defer e.Close() + + // Start a concurrent read operation in a busy loop. + readsBegun := make(chan struct{}) + readsDone := make(chan error) + writesDone := make(chan struct{}) + go func() { + readsDone <- func() error { + readsBegunAlias := readsBegun + for { + select { + case <-writesDone: + return nil + default: + val, err := mvccGetRawWithError(t, e, key) + if err != nil { + return err + } + if val != nil && !bytes.Equal(val, finalVal) { + return errors.Errorf("key value should be empty or %q; got %q", string(finalVal), string(val)) + } + if readsBegunAlias != nil { + close(readsBegunAlias) + readsBegunAlias = nil } - }() - }() - // Wait until we've succeeded with first read. - <-readsBegun - - // Create key/values and put them in a batch to engine. - batch := e.NewBatch() - defer batch.Close() - for i := 0; i < numWrites; i++ { - if err := batch.PutUnversioned(key.Key, []byte(strconv.Itoa(i))); err != nil { - t.Fatal(err) } } - if err := batch.Commit(false /* sync */); err != nil { - t.Fatal(err) - } - close(writesDone) - if err := <-readsDone; err != nil { - t.Fatal(err) - } - }) + }() + }() + // Wait until we've succeeded with first read. + <-readsBegun + + // Create key/values and put them in a batch to engine. + batch := e.NewBatch() + defer batch.Close() + for i := 0; i < numWrites; i++ { + if err := batch.PutUnversioned(key.Key, []byte(strconv.Itoa(i))); err != nil { + t.Fatal(err) + } + } + if err := batch.Commit(false /* sync */); err != nil { + t.Fatal(err) + } + close(writesDone) + if err := <-readsDone; err != nil { + t.Fatal(err) } } @@ -137,75 +132,71 @@ func TestEngineBatchStaleCachedIterator(t *testing.T) { // had since been deleted from the underlying engine. // Discovered in #6878. - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - eng := engineImpl.create() - defer eng.Close() - - // Focused failure mode: highlights the actual bug. - { - batch := eng.NewBatch() - defer batch.Close() - iter := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) - key := MVCCKey{Key: roachpb.Key("b")} - - if err := batch.PutUnversioned(key.Key, []byte("foo")); err != nil { - t.Fatal(err) - } + eng := NewDefaultInMemForTesting() + defer eng.Close() - iter.SeekGE(key) + // Focused failure mode: highlights the actual bug. + { + batch := eng.NewBatch() + defer batch.Close() + iter := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) + key := MVCCKey{Key: roachpb.Key("b")} - if err := batch.ClearUnversioned(key.Key); err != nil { - t.Fatal(err) - } + if err := batch.PutUnversioned(key.Key, []byte("foo")); err != nil { + t.Fatal(err) + } - // MVCCIterator should not reuse its cached result. - iter.SeekGE(key) + iter.SeekGE(key) - if ok, err := iter.Valid(); err != nil { - t.Fatal(err) - } else if ok { - t.Fatalf("iterator unexpectedly valid: %v -> %v", - iter.UnsafeKey(), iter.UnsafeValue()) - } + if err := batch.ClearUnversioned(key.Key); err != nil { + t.Fatal(err) + } - iter.Close() - } + // MVCCIterator should not reuse its cached result. + iter.SeekGE(key) - // Higher-level failure mode. Mostly for documentation. - { - batch := eng.NewBatch() - defer batch.Close() + if ok, err := iter.Valid(); err != nil { + t.Fatal(err) + } else if ok { + t.Fatalf("iterator unexpectedly valid: %v -> %v", + iter.UnsafeKey(), iter.UnsafeValue()) + } - key := roachpb.Key("z") + iter.Close() + } - // Put a value so that the deletion below finds a value to seek - // to. - if err := MVCCPut(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("x"), nil); err != nil { - t.Fatal(err) - } + // Higher-level failure mode. Mostly for documentation. + { + batch := eng.NewBatch() + defer batch.Close() - // Seek the iterator to `key` and clear the value (but without - // telling the iterator about that). - if _, err := MVCCDelete(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } + key := roachpb.Key("z") - // Trigger a seek on the cached iterator by seeking to the (now - // absent) key. - // The underlying iterator will already be in the right position - // due to a seek in MVCCDelete (followed by a Clear, which does not - // invalidate the iterator's cache), and if it reports its cached - // result back, we'll see the (newly deleted) value (due to the - // failure mode above). - if v, _, err := MVCCGet(context.Background(), batch, key, - hlc.Timestamp{}, MVCCGetOptions{}); err != nil { - t.Fatal(err) - } else if v != nil { - t.Fatalf("expected no value, got %+v", v) - } - } - }) + // Put a value so that the deletion below finds a value to seek + // to. + if err := MVCCPut(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("x"), nil); err != nil { + t.Fatal(err) + } + + // Seek the iterator to `key` and clear the value (but without + // telling the iterator about that). + if _, err := MVCCDelete(context.Background(), batch, nil, key, hlc.Timestamp{}, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } + + // Trigger a seek on the cached iterator by seeking to the (now + // absent) key. + // The underlying iterator will already be in the right position + // due to a seek in MVCCDelete (followed by a Clear, which does not + // invalidate the iterator's cache), and if it reports its cached + // result back, we'll see the (newly deleted) value (due to the + // failure mode above). + if v, _, err := MVCCGet(context.Background(), batch, key, + hlc.Timestamp{}, MVCCGetOptions{}); err != nil { + t.Fatal(err) + } else if v != nil { + t.Fatalf("expected no value, got %+v", v) + } } } @@ -213,139 +204,135 @@ func TestEngineBatch(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - numShuffles := 100 - key := mvccKey("a") - // Those are randomized below. - type data struct { - key MVCCKey - value []byte - merge bool + engine := NewDefaultInMemForTesting() + defer engine.Close() + + numShuffles := 100 + key := mvccKey("a") + // Those are randomized below. + type data struct { + key MVCCKey + value []byte + merge bool + } + batch := []data{ + {key, appender("~ockroachDB"), false}, + {key, appender("C~ckroachDB"), false}, + {key, appender("Co~kroachDB"), false}, + {key, appender("Coc~roachDB"), false}, + {key, appender("Cock~oachDB"), false}, + {key, appender("Cockr~achDB"), false}, + {key, appender("Cockro~chDB"), false}, + {key, appender("Cockroa~hDB"), false}, + {key, appender("Cockroac~DB"), false}, + {key, appender("Cockroach~B"), false}, + {key, appender("CockroachD~"), false}, + {key, nil, false}, + {key, appender("C"), true}, + {key, appender(" o"), true}, + {key, appender(" c"), true}, + {key, appender(" k"), true}, + {key, appender("r"), true}, + {key, appender(" o"), true}, + {key, appender(" a"), true}, + {key, appender(" c"), true}, + {key, appender("h"), true}, + {key, appender(" D"), true}, + {key, appender(" B"), true}, + } + + apply := func(rw ReadWriter, d data) error { + if d.value == nil { + return rw.ClearUnversioned(d.key.Key) + } else if d.merge { + return rw.Merge(d.key, d.value) + } + return rw.PutUnversioned(d.key.Key, d.value) + } + + get := func(rw ReadWriter, key MVCCKey) []byte { + b := mvccGetRaw(t, rw, key) + var m enginepb.MVCCMetadata + if err := protoutil.Unmarshal(b, &m); err != nil { + t.Fatal(err) + } + if !m.IsInline() { + return nil + } + valueBytes, err := MakeValue(m).GetBytes() + if err != nil { + t.Fatal(err) + } + return valueBytes + } + + for i := 0; i < numShuffles; i++ { + // In each run, create an array of shuffled operations. + shuffledIndices := rand.Perm(len(batch)) + currentBatch := make([]data, len(batch)) + for k := range currentBatch { + currentBatch[k] = batch[shuffledIndices[k]] + } + // Reset the key + if err := engine.ClearUnversioned(key.Key); err != nil { + t.Fatal(err) + } + // Run it once with individual operations and remember the result. + for i, op := range currentBatch { + if err := apply(engine, op); err != nil { + t.Errorf("%d: op %v: %+v", i, op, err) + continue } - batch := []data{ - {key, appender("~ockroachDB"), false}, - {key, appender("C~ckroachDB"), false}, - {key, appender("Co~kroachDB"), false}, - {key, appender("Coc~roachDB"), false}, - {key, appender("Cock~oachDB"), false}, - {key, appender("Cockr~achDB"), false}, - {key, appender("Cockro~chDB"), false}, - {key, appender("Cockroa~hDB"), false}, - {key, appender("Cockroac~DB"), false}, - {key, appender("Cockroach~B"), false}, - {key, appender("CockroachD~"), false}, - {key, nil, false}, - {key, appender("C"), true}, - {key, appender(" o"), true}, - {key, appender(" c"), true}, - {key, appender(" k"), true}, - {key, appender("r"), true}, - {key, appender(" o"), true}, - {key, appender(" a"), true}, - {key, appender(" c"), true}, - {key, appender("h"), true}, - {key, appender(" D"), true}, - {key, appender(" B"), true}, + } + expectedValue := get(engine, key) + // Run the whole thing as a batch and compare. + b := engine.NewBatch() + defer b.Close() + if err := b.ClearUnversioned(key.Key); err != nil { + t.Fatal(err) + } + for _, op := range currentBatch { + if err := apply(b, op); err != nil { + t.Fatal(err) } - - apply := func(rw ReadWriter, d data) error { - if d.value == nil { - return rw.ClearUnversioned(d.key.Key) - } else if d.merge { - return rw.Merge(d.key, d.value) - } - return rw.PutUnversioned(d.key.Key, d.value) + } + // Try getting the value from the batch. + actualValue := get(b, key) + if !bytes.Equal(actualValue, expectedValue) { + t.Errorf("%d: expected %s, but got %s", i, expectedValue, actualValue) + } + // Try using an iterator to get the value from the batch. + iter := b.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) + iter.SeekGE(key) + if ok, err := iter.Valid(); !ok { + if currentBatch[len(currentBatch)-1].value != nil { + t.Errorf("%d: batch seek invalid, err=%v", i, err) } - - get := func(rw ReadWriter, key MVCCKey) []byte { - b := mvccGetRaw(t, rw, key) - var m enginepb.MVCCMetadata - if err := protoutil.Unmarshal(b, &m); err != nil { - t.Fatal(err) - } - if !m.IsInline() { - return nil - } - valueBytes, err := MakeValue(m).GetBytes() - if err != nil { - t.Fatal(err) - } - return valueBytes + } else if !iter.Key().Equal(key) { + t.Errorf("%d: batch seek expected key %s, but got %s", i, key, iter.Key()) + } else { + var m enginepb.MVCCMetadata + if err := iter.ValueProto(&m); err != nil { + t.Fatal(err) } - - for i := 0; i < numShuffles; i++ { - // In each run, create an array of shuffled operations. - shuffledIndices := rand.Perm(len(batch)) - currentBatch := make([]data, len(batch)) - for k := range currentBatch { - currentBatch[k] = batch[shuffledIndices[k]] - } - // Reset the key - if err := engine.ClearUnversioned(key.Key); err != nil { - t.Fatal(err) - } - // Run it once with individual operations and remember the result. - for i, op := range currentBatch { - if err := apply(engine, op); err != nil { - t.Errorf("%d: op %v: %+v", i, op, err) - continue - } - } - expectedValue := get(engine, key) - // Run the whole thing as a batch and compare. - b := engine.NewBatch() - defer b.Close() - if err := b.ClearUnversioned(key.Key); err != nil { - t.Fatal(err) - } - for _, op := range currentBatch { - if err := apply(b, op); err != nil { - t.Fatal(err) - } - } - // Try getting the value from the batch. - actualValue := get(b, key) - if !bytes.Equal(actualValue, expectedValue) { - t.Errorf("%d: expected %s, but got %s", i, expectedValue, actualValue) - } - // Try using an iterator to get the value from the batch. - iter := b.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) - iter.SeekGE(key) - if ok, err := iter.Valid(); !ok { - if currentBatch[len(currentBatch)-1].value != nil { - t.Errorf("%d: batch seek invalid, err=%v", i, err) - } - } else if !iter.Key().Equal(key) { - t.Errorf("%d: batch seek expected key %s, but got %s", i, key, iter.Key()) - } else { - var m enginepb.MVCCMetadata - if err := iter.ValueProto(&m); err != nil { - t.Fatal(err) - } - valueBytes, err := MakeValue(m).GetBytes() - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(valueBytes, expectedValue) { - t.Errorf("%d: expected %s, but got %s", i, expectedValue, valueBytes) - } - } - iter.Close() - // Commit the batch and try getting the value from the engine. - if err := b.Commit(false /* sync */); err != nil { - t.Errorf("%d: %+v", i, err) - continue - } - actualValue = get(engine, key) - if !bytes.Equal(actualValue, expectedValue) { - t.Errorf("%d: expected %s, but got %s", i, expectedValue, actualValue) - } + valueBytes, err := MakeValue(m).GetBytes() + if err != nil { + t.Fatal(err) } - }) + if !bytes.Equal(valueBytes, expectedValue) { + t.Errorf("%d: expected %s, but got %s", i, expectedValue, valueBytes) + } + } + iter.Close() + // Commit the batch and try getting the value from the engine. + if err := b.Commit(false /* sync */); err != nil { + t.Errorf("%d: %+v", i, err) + continue + } + actualValue = get(engine, key) + if !bytes.Equal(actualValue, expectedValue) { + t.Errorf("%d: expected %s, but got %s", i, expectedValue, actualValue) + } } } @@ -353,56 +340,52 @@ func TestEnginePutGetDelete(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - // Test for correct handling of empty keys, which should produce errors. - for i, err := range []error{ - engine.PutUnversioned(mvccKey("").Key, []byte("")), - engine.PutUnversioned(NilKey.Key, []byte("")), - engine.ClearUnversioned(NilKey.Key), - engine.ClearUnversioned(NilKey.Key), - engine.ClearUnversioned(mvccKey("").Key), - } { - if err == nil { - t.Fatalf("%d: illegal handling of empty key", i) - } - } + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Test for allowed keys, which should go through. - testCases := []struct { - key MVCCKey - value []byte - }{ - {mvccKey("dog"), []byte("woof")}, - {mvccKey("cat"), []byte("meow")}, - {mvccKey("emptyval"), nil}, - {mvccKey("emptyval2"), []byte("")}, - {mvccKey("server"), []byte("42")}, - } - for _, c := range testCases { - val := mvccGetRaw(t, engine, c.key) - if len(val) != 0 { - t.Errorf("expected key %q value.Bytes to be nil: got %+v", c.key, val) - } - if err := engine.PutUnversioned(c.key.Key, c.value); err != nil { - t.Errorf("put: expected no error, but got %s", err) - } - val = mvccGetRaw(t, engine, c.key) - if !bytes.Equal(val, c.value) { - t.Errorf("expected key value %s to be %+v: got %+v", c.key, c.value, val) - } - if err := engine.ClearUnversioned(c.key.Key); err != nil { - t.Errorf("delete: expected no error, but got %s", err) - } - val = mvccGetRaw(t, engine, c.key) - if len(val) != 0 { - t.Errorf("expected key %s value.Bytes to be nil: got %+v", c.key, val) - } - } - }) + // Test for correct handling of empty keys, which should produce errors. + for i, err := range []error{ + engine.PutUnversioned(mvccKey("").Key, []byte("")), + engine.PutUnversioned(NilKey.Key, []byte("")), + engine.ClearUnversioned(NilKey.Key), + engine.ClearUnversioned(NilKey.Key), + engine.ClearUnversioned(mvccKey("").Key), + } { + if err == nil { + t.Fatalf("%d: illegal handling of empty key", i) + } + } + + // Test for allowed keys, which should go through. + testCases := []struct { + key MVCCKey + value []byte + }{ + {mvccKey("dog"), []byte("woof")}, + {mvccKey("cat"), []byte("meow")}, + {mvccKey("emptyval"), nil}, + {mvccKey("emptyval2"), []byte("")}, + {mvccKey("server"), []byte("42")}, + } + for _, c := range testCases { + val := mvccGetRaw(t, engine, c.key) + if len(val) != 0 { + t.Errorf("expected key %q value.Bytes to be nil: got %+v", c.key, val) + } + if err := engine.PutUnversioned(c.key.Key, c.value); err != nil { + t.Errorf("put: expected no error, but got %s", err) + } + val = mvccGetRaw(t, engine, c.key) + if !bytes.Equal(val, c.value) { + t.Errorf("expected key value %s to be %+v: got %+v", c.key, c.value, val) + } + if err := engine.ClearUnversioned(c.key.Key); err != nil { + t.Errorf("delete: expected no error, but got %s", err) + } + val = mvccGetRaw(t, engine, c.key) + if len(val) != 0 { + t.Errorf("expected key %s value.Bytes to be nil: got %+v", c.key, val) + } } } @@ -465,91 +448,71 @@ func TestEngineMerge(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - engineBytes := make([][][]byte, len(mvccEngineImpls)) - for engineIndex, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - testcases := []struct { - testKey MVCCKey - merges [][]byte - expected []byte - }{ - { - // Test case with RawBytes only. - mvccKey("haste not in life"), - [][]byte{ - appender("x"), - appender("y"), - appender("z"), - }, - appender("xyz"), - }, - { - // Test case with RawBytes and MergeTimestamp. - mvccKey("timeseriesmerged"), - [][]byte{ - addMergeTimestamp(t, timeSeriesRow(testtime, 1000, []tsSample{ - {1, 1, 5, 5, 5}, - }...), 27), - timeSeriesRow(testtime, 1000, []tsSample{ - {2, 1, 5, 5, 5}, - {1, 2, 10, 7, 3}, - }...), - addMergeTimestamp(t, timeSeriesRow(testtime, 1000, []tsSample{ - {10, 1, 5, 5, 5}, - }...), 53), - timeSeriesRow(testtime, 1000, []tsSample{ - {5, 1, 5, 5, 5}, - {3, 1, 5, 5, 5}, - }...), - }, - addMergeTimestamp(t, timeSeriesRow(testtime, 1000, []tsSample{ - {1, 2, 10, 7, 3}, - {2, 1, 5, 5, 5}, - {3, 1, 5, 5, 5}, - {5, 1, 5, 5, 5}, - {10, 1, 5, 5, 5}, - }...), 27), - }, - } - engineBytes[engineIndex] = make([][]byte, len(testcases)) - for tcIndex, tc := range testcases { - for i, update := range tc.merges { - if err := engine.Merge(tc.testKey, update); err != nil { - t.Fatalf("%d: %+v", i, err) - } - } - result := mvccGetRaw(t, engine, tc.testKey) - engineBytes[engineIndex][tcIndex] = result - var resultV, expectedV enginepb.MVCCMetadata - if err := protoutil.Unmarshal(result, &resultV); err != nil { - t.Fatal(err) - } - if err := protoutil.Unmarshal(tc.expected, &expectedV); err != nil { - t.Fatal(err) - } - if !reflect.DeepEqual(resultV, expectedV) { - t.Errorf("unexpected append-merge result: %v != %v", resultV, expectedV) - } - } - }) + engine := NewDefaultInMemForTesting() + defer engine.Close() + + testcases := []struct { + testKey MVCCKey + merges [][]byte + expected []byte + }{ + { + // Test case with RawBytes only. + mvccKey("haste not in life"), + [][]byte{ + appender("x"), + appender("y"), + appender("z"), + }, + appender("xyz"), + }, + { + // Test case with RawBytes and MergeTimestamp. + mvccKey("timeseriesmerged"), + [][]byte{ + addMergeTimestamp(t, timeSeriesRow(testtime, 1000, []tsSample{ + {1, 1, 5, 5, 5}, + }...), 27), + timeSeriesRow(testtime, 1000, []tsSample{ + {2, 1, 5, 5, 5}, + {1, 2, 10, 7, 3}, + }...), + addMergeTimestamp(t, timeSeriesRow(testtime, 1000, []tsSample{ + {10, 1, 5, 5, 5}, + }...), 53), + timeSeriesRow(testtime, 1000, []tsSample{ + {5, 1, 5, 5, 5}, + {3, 1, 5, 5, 5}, + }...), + }, + addMergeTimestamp(t, timeSeriesRow(testtime, 1000, []tsSample{ + {1, 2, 10, 7, 3}, + {2, 1, 5, 5, 5}, + {3, 1, 5, 5, 5}, + {5, 1, 5, 5, 5}, + {10, 1, 5, 5, 5}, + }...), 27), + }, } - for i := 0; i < len(engineBytes); i++ { - // Pair-wise comparison of bytes since difference in serialization - // can trigger replica consistency checker failures #45811 - if i+1 == len(engineBytes) { - break - } - eng1 := i - eng2 := i + 1 - for j := 0; j < len(engineBytes[eng1]); j++ { - if !bytes.Equal(engineBytes[eng1][j], engineBytes[eng2][j]) { - t.Errorf("engines %d, %d differ at test %d:\n%s\n != \n%s\n", eng1, eng2, j, - hex.Dump(engineBytes[eng1][j]), hex.Dump(engineBytes[eng2][j])) + engineBytes := make([][]byte, len(testcases)) + for tcIndex, tc := range testcases { + for i, update := range tc.merges { + if err := engine.Merge(tc.testKey, update); err != nil { + t.Fatalf("%d: %+v", i, err) } } + result := mvccGetRaw(t, engine, tc.testKey) + engineBytes[tcIndex] = result + var resultV, expectedV enginepb.MVCCMetadata + if err := protoutil.Unmarshal(result, &resultV); err != nil { + t.Fatal(err) + } + if err := protoutil.Unmarshal(tc.expected, &expectedV); err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(resultV, expectedV) { + t.Errorf("unexpected append-merge result: %v != %v", resultV, expectedV) + } } } @@ -701,37 +664,33 @@ func TestFlushNumSSTables(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - batch := engine.NewBatch() - for i := 0; i < 10000; i++ { - key := make([]byte, 4) - binary.BigEndian.PutUint32(key, uint32(i)) - err := batch.PutUnversioned(key, []byte("foobar")) - if err != nil { - t.Fatal(err) - } - } + engine := NewDefaultInMemForTesting() + defer engine.Close() - err := batch.Commit(true) - if err != nil { - t.Fatal(err) - } - batch.Close() + batch := engine.NewBatch() + for i := 0; i < 10000; i++ { + key := make([]byte, 4) + binary.BigEndian.PutUint32(key, uint32(i)) + err := batch.PutUnversioned(key, []byte("foobar")) + if err != nil { + t.Fatal(err) + } + } - err = engine.Flush() - if err != nil { - t.Fatal(err) - } + err := batch.Commit(true) + if err != nil { + t.Fatal(err) + } + batch.Close() - m := engine.GetMetrics() - if m.NumSSTables() == 0 { - t.Fatal("expected non-zero sstables, got 0") - } - }) + err = engine.Flush() + if err != nil { + t.Fatal(err) + } + + m := engine.GetMetrics() + if m.NumSSTables() == 0 { + t.Fatal("expected non-zero sstables, got 0") } } @@ -739,103 +698,99 @@ func TestEngineScan1(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - testCases := []struct { - key MVCCKey - value []byte - }{ - {mvccKey("dog"), []byte("woof")}, - {mvccKey("cat"), []byte("meow")}, - {mvccKey("server"), []byte("42")}, - {mvccKey("french"), []byte("Allô?")}, - {mvccKey("german"), []byte("hallo")}, - {mvccKey("chinese"), []byte("你好")}, - } - keyMap := map[string][]byte{} - for _, c := range testCases { - if err := engine.PutUnversioned(c.key.Key, c.value); err != nil { - t.Errorf("could not put key %q: %+v", c.key, err) - } - keyMap[string(c.key.Key)] = c.value - } - sortedKeys := make([]string, len(testCases)) - for i, t := range testCases { - sortedKeys[i] = string(t.key.Key) - } - sort.Strings(sortedKeys) + engine := NewDefaultInMemForTesting() + defer engine.Close() - keyvals, err := Scan(engine, roachpb.Key("chinese"), roachpb.Key("german"), 0) - if err != nil { - t.Fatalf("could not run scan: %+v", err) - } - ensureRangeEqual(t, sortedKeys[1:4], keyMap, keyvals) + testCases := []struct { + key MVCCKey + value []byte + }{ + {mvccKey("dog"), []byte("woof")}, + {mvccKey("cat"), []byte("meow")}, + {mvccKey("server"), []byte("42")}, + {mvccKey("french"), []byte("Allô?")}, + {mvccKey("german"), []byte("hallo")}, + {mvccKey("chinese"), []byte("你好")}, + } + keyMap := map[string][]byte{} + for _, c := range testCases { + if err := engine.PutUnversioned(c.key.Key, c.value); err != nil { + t.Errorf("could not put key %q: %+v", c.key, err) + } + keyMap[string(c.key.Key)] = c.value + } + sortedKeys := make([]string, len(testCases)) + for i, t := range testCases { + sortedKeys[i] = string(t.key.Key) + } + sort.Strings(sortedKeys) - // Check an end of range which does not equal an existing key. - keyvals, err = Scan(engine, roachpb.Key("chinese"), roachpb.Key("german1"), 0) - if err != nil { - t.Fatalf("could not run scan: %+v", err) - } - ensureRangeEqual(t, sortedKeys[1:5], keyMap, keyvals) + keyvals, err := Scan(engine, roachpb.Key("chinese"), roachpb.Key("german"), 0) + if err != nil { + t.Fatalf("could not run scan: %+v", err) + } + ensureRangeEqual(t, sortedKeys[1:4], keyMap, keyvals) - keyvals, err = Scan(engine, roachpb.Key("chinese"), roachpb.Key("german"), 2) - if err != nil { - t.Fatalf("could not run scan: %+v", err) - } - ensureRangeEqual(t, sortedKeys[1:3], keyMap, keyvals) - - // Should return all key/value pairs in lexicographic order. Note that - // LocalMax is the lowest possible global key. - startKeys := []roachpb.Key{roachpb.Key("cat"), keys.LocalMax} - for _, startKey := range startKeys { - keyvals, err = Scan(engine, startKey, roachpb.KeyMax, 0) - if err != nil { - t.Fatalf("could not run scan: %+v", err) - } - ensureRangeEqual(t, sortedKeys, keyMap, keyvals) - } + // Check an end of range which does not equal an existing key. + keyvals, err = Scan(engine, roachpb.Key("chinese"), roachpb.Key("german1"), 0) + if err != nil { + t.Fatalf("could not run scan: %+v", err) + } + ensureRangeEqual(t, sortedKeys[1:5], keyMap, keyvals) - // Test iterator stats. - ro := engine.NewReadOnly(StandardDurability) - iter := ro.NewMVCCIterator(MVCCKeyIterKind, - IterOptions{LowerBound: roachpb.Key("cat"), UpperBound: roachpb.Key("server")}) - iter.SeekGE(MVCCKey{Key: roachpb.Key("cat")}) - for { - valid, err := iter.Valid() - require.NoError(t, err) - if !valid { - break - } - iter.Next() - } - stats := iter.Stats().Stats - // Setting non-deterministic InternalStats to empty. - stats.InternalStats = pebble.InternalIteratorStats{} - require.Equal(t, "(interface (dir, seek, step): (fwd, 1, 5), (rev, 0, 0)), "+ - "(internal (dir, seek, step): (fwd, 1, 5), (rev, 0, 0))", stats.String()) - iter.Close() - iter = ro.NewMVCCIterator(MVCCKeyIterKind, - IterOptions{LowerBound: roachpb.Key("cat"), UpperBound: roachpb.Key("server")}) - // pebble.Iterator is reused, but stats are reset. - stats = iter.Stats().Stats - // Setting non-deterministic InternalStats to empty. - stats.InternalStats = pebble.InternalIteratorStats{} - require.Equal(t, "(interface (dir, seek, step): (fwd, 0, 0), (rev, 0, 0)), "+ - "(internal (dir, seek, step): (fwd, 0, 0), (rev, 0, 0))", stats.String()) - iter.SeekGE(MVCCKey{Key: roachpb.Key("french")}) - iter.SeekLT(MVCCKey{Key: roachpb.Key("server")}) - stats = iter.Stats().Stats - // Setting non-deterministic InternalStats to empty. - stats.InternalStats = pebble.InternalIteratorStats{} - require.Equal(t, "(interface (dir, seek, step): (fwd, 1, 0), (rev, 1, 0)), "+ - "(internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 1))", stats.String()) - iter.Close() - ro.Close() - }) + keyvals, err = Scan(engine, roachpb.Key("chinese"), roachpb.Key("german"), 2) + if err != nil { + t.Fatalf("could not run scan: %+v", err) + } + ensureRangeEqual(t, sortedKeys[1:3], keyMap, keyvals) + + // Should return all key/value pairs in lexicographic order. Note that + // LocalMax is the lowest possible global key. + startKeys := []roachpb.Key{roachpb.Key("cat"), keys.LocalMax} + for _, startKey := range startKeys { + keyvals, err = Scan(engine, startKey, roachpb.KeyMax, 0) + if err != nil { + t.Fatalf("could not run scan: %+v", err) + } + ensureRangeEqual(t, sortedKeys, keyMap, keyvals) + } + + // Test iterator stats. + ro := engine.NewReadOnly(StandardDurability) + iter := ro.NewMVCCIterator(MVCCKeyIterKind, + IterOptions{LowerBound: roachpb.Key("cat"), UpperBound: roachpb.Key("server")}) + iter.SeekGE(MVCCKey{Key: roachpb.Key("cat")}) + for { + valid, err := iter.Valid() + require.NoError(t, err) + if !valid { + break + } + iter.Next() } + stats := iter.Stats().Stats + // Setting non-deterministic InternalStats to empty. + stats.InternalStats = pebble.InternalIteratorStats{} + require.Equal(t, "(interface (dir, seek, step): (fwd, 1, 5), (rev, 0, 0)), "+ + "(internal (dir, seek, step): (fwd, 1, 5), (rev, 0, 0))", stats.String()) + iter.Close() + iter = ro.NewMVCCIterator(MVCCKeyIterKind, + IterOptions{LowerBound: roachpb.Key("cat"), UpperBound: roachpb.Key("server")}) + // pebble.Iterator is reused, but stats are reset. + stats = iter.Stats().Stats + // Setting non-deterministic InternalStats to empty. + stats.InternalStats = pebble.InternalIteratorStats{} + require.Equal(t, "(interface (dir, seek, step): (fwd, 0, 0), (rev, 0, 0)), "+ + "(internal (dir, seek, step): (fwd, 0, 0), (rev, 0, 0))", stats.String()) + iter.SeekGE(MVCCKey{Key: roachpb.Key("french")}) + iter.SeekLT(MVCCKey{Key: roachpb.Key("server")}) + stats = iter.Stats().Stats + // Setting non-deterministic InternalStats to empty. + stats.InternalStats = pebble.InternalIteratorStats{} + require.Equal(t, "(interface (dir, seek, step): (fwd, 1, 0), (rev, 1, 0)), "+ + "(internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 1))", stats.String()) + iter.Close() + ro.Close() } func verifyScan(start, end roachpb.Key, max int64, expKeys []MVCCKey, engine Engine, t *testing.T) { @@ -860,92 +815,84 @@ func TestEngineScan2(t *testing.T) { // TODO(Tobias): Merge this with TestEngineScan1 and remove // either verifyScan or the other helper function. - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - keys := []MVCCKey{ - mvccKey("a"), - mvccKey("aa"), - mvccKey("aaa"), - mvccKey("ab"), - mvccKey("abc"), - mvccKey(roachpb.RKeyMax), - } + engine := NewDefaultInMemForTesting() + defer engine.Close() - insertKeys(keys, engine, t) + keys := []MVCCKey{ + mvccKey("a"), + mvccKey("aa"), + mvccKey("aaa"), + mvccKey("ab"), + mvccKey("abc"), + mvccKey(roachpb.RKeyMax), + } - // Scan all keys (non-inclusive of final key). - verifyScan(localMax, roachpb.KeyMax, 10, keys[:5], engine, t) - verifyScan(roachpb.Key("a"), roachpb.KeyMax, 10, keys[:5], engine, t) + insertKeys(keys, engine, t) - // Scan sub range. - verifyScan(roachpb.Key("aab"), roachpb.Key("abcc"), 10, keys[3:5], engine, t) - verifyScan(roachpb.Key("aa0"), roachpb.Key("abcc"), 10, keys[2:5], engine, t) + // Scan all keys (non-inclusive of final key). + verifyScan(localMax, roachpb.KeyMax, 10, keys[:5], engine, t) + verifyScan(roachpb.Key("a"), roachpb.KeyMax, 10, keys[:5], engine, t) - // Scan with max values. - verifyScan(localMax, roachpb.KeyMax, 3, keys[:3], engine, t) - verifyScan(roachpb.Key("a0"), roachpb.KeyMax, 3, keys[1:4], engine, t) + // Scan sub range. + verifyScan(roachpb.Key("aab"), roachpb.Key("abcc"), 10, keys[3:5], engine, t) + verifyScan(roachpb.Key("aa0"), roachpb.Key("abcc"), 10, keys[2:5], engine, t) - // Scan with max value 0 gets all values. - verifyScan(localMax, roachpb.KeyMax, 0, keys[:5], engine, t) - }) - } + // Scan with max values. + verifyScan(localMax, roachpb.KeyMax, 3, keys[:3], engine, t) + verifyScan(roachpb.Key("a0"), roachpb.KeyMax, 3, keys[1:4], engine, t) + + // Scan with max value 0 gets all values. + verifyScan(localMax, roachpb.KeyMax, 0, keys[:5], engine, t) } func TestSnapshot(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - key := mvccKey("a") - val1 := []byte("1") - if err := engine.PutUnversioned(key.Key, val1); err != nil { - t.Fatal(err) - } - val := mvccGetRaw(t, engine, key) - if !bytes.Equal(val, val1) { - t.Fatalf("the value %s in get result does not match the value %s in request", - val, val1) - } + key := mvccKey("a") + val1 := []byte("1") + if err := engine.PutUnversioned(key.Key, val1); err != nil { + t.Fatal(err) + } + val := mvccGetRaw(t, engine, key) + if !bytes.Equal(val, val1) { + t.Fatalf("the value %s in get result does not match the value %s in request", + val, val1) + } - snap := engine.NewSnapshot() - defer snap.Close() + snap := engine.NewSnapshot() + defer snap.Close() - val2 := []byte("2") - if err := engine.PutUnversioned(key.Key, val2); err != nil { - t.Fatal(err) - } - val = mvccGetRaw(t, engine, key) - valSnapshot := mvccGetRaw(t, snap, key) - if !bytes.Equal(val, val2) { - t.Fatalf("the value %s in get result does not match the value %s in request", - val, val2) - } - if !bytes.Equal(valSnapshot, val1) { - t.Fatalf("the value %s in get result does not match the value %s in request", - valSnapshot, val1) - } + val2 := []byte("2") + if err := engine.PutUnversioned(key.Key, val2); err != nil { + t.Fatal(err) + } + val = mvccGetRaw(t, engine, key) + valSnapshot := mvccGetRaw(t, snap, key) + if !bytes.Equal(val, val2) { + t.Fatalf("the value %s in get result does not match the value %s in request", + val, val2) + } + if !bytes.Equal(valSnapshot, val1) { + t.Fatalf("the value %s in get result does not match the value %s in request", + valSnapshot, val1) + } - keyvals, _ := Scan(engine, key.Key, roachpb.KeyMax, 0) - keyvalsSnapshot, error := Scan(snap, key.Key, roachpb.KeyMax, 0) - if error != nil { - t.Fatalf("error : %s", error) - } - if len(keyvals) != 1 || !bytes.Equal(keyvals[0].Value, val2) { - t.Fatalf("the value %s in get result does not match the value %s in request", - keyvals[0].Value, val2) - } - if len(keyvalsSnapshot) != 1 || !bytes.Equal(keyvalsSnapshot[0].Value, val1) { - t.Fatalf("the value %s in get result does not match the value %s in request", - keyvalsSnapshot[0].Value, val1) - } - }) + keyvals, _ := Scan(engine, key.Key, roachpb.KeyMax, 0) + keyvalsSnapshot, error := Scan(snap, key.Key, roachpb.KeyMax, 0) + if error != nil { + t.Fatalf("error : %s", error) + } + if len(keyvals) != 1 || !bytes.Equal(keyvals[0].Value, val2) { + t.Fatalf("the value %s in get result does not match the value %s in request", + keyvals[0].Value, val2) + } + if len(keyvalsSnapshot) != 1 || !bytes.Equal(keyvalsSnapshot[0].Value, val1) { + t.Fatalf("the value %s in get result does not match the value %s in request", + keyvalsSnapshot[0].Value, val1) } } @@ -955,77 +902,73 @@ func TestSnapshotMethods(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - keys := []MVCCKey{mvccKey("a"), mvccKey("b")} - vals := [][]byte{[]byte("1"), []byte("2")} - for i := range keys { - if err := engine.PutUnversioned(keys[i].Key, vals[i]); err != nil { - t.Fatal(err) - } - val := mvccGetRaw(t, engine, keys[i]) - if !bytes.Equal(vals[i], val) { - t.Fatalf("expected %s, but found %s", vals[i], val) - } - } - snap := engine.NewSnapshot() - defer snap.Close() - - // Verify Get. - for i := range keys { - valSnapshot := mvccGetRaw(t, snap, keys[i]) - if !bytes.Equal(vals[i], valSnapshot) { - t.Fatalf("the value %s in get result does not match the value %s in snapshot", - vals[i], valSnapshot) - } - } + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Verify Scan. - keyvals, _ := Scan(engine, localMax, roachpb.KeyMax, 0) - keyvalsSnapshot, err := Scan(snap, localMax, roachpb.KeyMax, 0) - if err != nil { - t.Fatal(err) - } - if !reflect.DeepEqual(keyvals, keyvalsSnapshot) { - t.Fatalf("the key/values %v in scan result does not match the value %s in snapshot", - keyvals, keyvalsSnapshot) - } + keys := []MVCCKey{mvccKey("a"), mvccKey("b")} + vals := [][]byte{[]byte("1"), []byte("2")} + for i := range keys { + if err := engine.PutUnversioned(keys[i].Key, vals[i]); err != nil { + t.Fatal(err) + } + val := mvccGetRaw(t, engine, keys[i]) + if !bytes.Equal(vals[i], val) { + t.Fatalf("expected %s, but found %s", vals[i], val) + } + } + snap := engine.NewSnapshot() + defer snap.Close() + + // Verify Get. + for i := range keys { + valSnapshot := mvccGetRaw(t, snap, keys[i]) + if !bytes.Equal(vals[i], valSnapshot) { + t.Fatalf("the value %s in get result does not match the value %s in snapshot", + vals[i], valSnapshot) + } + } - // Verify MVCCIterate. - index := 0 - if err := snap.MVCCIterate(localMax, roachpb.KeyMax, MVCCKeyAndIntentsIterKind, IterKeyTypePointsOnly, - func(kv MVCCKeyValue, _ MVCCRangeKeyStack) error { - if !kv.Key.Equal(keys[index]) || !bytes.Equal(kv.Value, vals[index]) { - t.Errorf("%d: key/value not equal between expected and snapshot: %s/%s, %s/%s", - index, keys[index], vals[index], kv.Key, kv.Value) - } - index++ - return nil - }); err != nil { - t.Fatal(err) - } + // Verify Scan. + keyvals, _ := Scan(engine, localMax, roachpb.KeyMax, 0) + keyvalsSnapshot, err := Scan(snap, localMax, roachpb.KeyMax, 0) + if err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(keyvals, keyvalsSnapshot) { + t.Fatalf("the key/values %v in scan result does not match the value %s in snapshot", + keyvals, keyvalsSnapshot) + } - // Write a new key to engine. - newKey := mvccKey("c") - newVal := []byte("3") - if err := engine.PutUnversioned(newKey.Key, newVal); err != nil { - t.Fatal(err) + // Verify MVCCIterate. + index := 0 + if err := snap.MVCCIterate(localMax, roachpb.KeyMax, MVCCKeyAndIntentsIterKind, IterKeyTypePointsOnly, + func(kv MVCCKeyValue, _ MVCCRangeKeyStack) error { + if !kv.Key.Equal(keys[index]) || !bytes.Equal(kv.Value, vals[index]) { + t.Errorf("%d: key/value not equal between expected and snapshot: %s/%s, %s/%s", + index, keys[index], vals[index], kv.Key, kv.Value) } + index++ + return nil + }); err != nil { + t.Fatal(err) + } - // Verify NewMVCCIterator still iterates over original snapshot. - iter := snap.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) - iter.SeekGE(newKey) - if ok, err := iter.Valid(); err != nil { - t.Fatal(err) - } else if ok { - t.Error("expected invalid iterator when seeking to element which shouldn't be visible to snapshot") - } - iter.Close() - }) + // Write a new key to engine. + newKey := mvccKey("c") + newVal := []byte("3") + if err := engine.PutUnversioned(newKey.Key, newVal); err != nil { + t.Fatal(err) + } + + // Verify NewMVCCIterator still iterates over original snapshot. + iter := snap.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) + iter.SeekGE(newKey) + if ok, err := iter.Valid(); err != nil { + t.Fatal(err) + } else if ok { + t.Error("expected invalid iterator when seeking to element which shouldn't be visible to snapshot") } + iter.Close() } func insertKeys(keys []MVCCKey, engine Engine, t *testing.T) { @@ -1113,164 +1056,160 @@ func TestEngineFS(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() - - testCases := []string{ - "1a: f = create /bar", - "1b: f.write abcdefghijklmnopqrstuvwxyz", - "1c: f.close", - "2a: f = open /bar", - "2b: f.read 5 == abcde", - "2c: f.readat 2 1 == bc", - "2d: f.readat 5 20 == uvwxy", - "2e: f.close", - "3a: link /bar /baz", - "3b: f = open /baz", - "3c: f.read 5 == abcde", - "3d: f.close", - "4a: delete /bar", - "4b: f = open /baz", - "4c: f.read 5 == abcde", - "4d: f.close", - "4e: open /bar [does-not-exist]", - "5a: rename /baz /foo", - "5b: f = open /foo", - "5c: f.readat 5 20 == uvwxy", - "5d: f.close", - "5e: open /baz [does-not-exist]", - "6a: f = create /red", - "6b: f.write blue", - "6c: f.sync", - "6d: f.close", - "7a: f = opendir /", - "7b: f.sync", - "7c: f.close", - "8a: f = create-with-sync /bar", - "8b: f.write ghe", - "8c: f.close", - "8d: f = open /bar", - "8e: f.read 3 == ghe", - "9a: create-dir /dir1", - "9b: create /dir1/bar", - "9c: list-dir /dir1 == bar", - "9d: create /dir1/baz", - "9e: list-dir /dir1 == bar,baz", - "9f: delete /dir1/bar", - "9g: delete /dir1/baz", - "9h: delete /dir1", - } + e := NewDefaultInMemForTesting() + defer e.Close() + + testCases := []string{ + "1a: f = create /bar", + "1b: f.write abcdefghijklmnopqrstuvwxyz", + "1c: f.close", + "2a: f = open /bar", + "2b: f.read 5 == abcde", + "2c: f.readat 2 1 == bc", + "2d: f.readat 5 20 == uvwxy", + "2e: f.close", + "3a: link /bar /baz", + "3b: f = open /baz", + "3c: f.read 5 == abcde", + "3d: f.close", + "4a: delete /bar", + "4b: f = open /baz", + "4c: f.read 5 == abcde", + "4d: f.close", + "4e: open /bar [does-not-exist]", + "5a: rename /baz /foo", + "5b: f = open /foo", + "5c: f.readat 5 20 == uvwxy", + "5d: f.close", + "5e: open /baz [does-not-exist]", + "6a: f = create /red", + "6b: f.write blue", + "6c: f.sync", + "6d: f.close", + "7a: f = opendir /", + "7b: f.sync", + "7c: f.close", + "8a: f = create-with-sync /bar", + "8b: f.write ghe", + "8c: f.close", + "8d: f = open /bar", + "8e: f.read 3 == ghe", + "9a: create-dir /dir1", + "9b: create /dir1/bar", + "9c: list-dir /dir1 == bar", + "9d: create /dir1/baz", + "9e: list-dir /dir1 == bar,baz", + "9f: delete /dir1/bar", + "9g: delete /dir1/baz", + "9h: delete /dir1", + } - var f fs.File - for _, tc := range testCases { - s := strings.Split(tc, " ")[1:] + var f fs.File + for _, tc := range testCases { + s := strings.Split(tc, " ")[1:] - saveF := s[0] == "f" && s[1] == "=" - if saveF { - s = s[2:] - } + saveF := s[0] == "f" && s[1] == "=" + if saveF { + s = s[2:] + } - fails := s[len(s)-1][0] == '[' - var errorStr string - if fails { - errorStr = s[len(s)-1][1:] - errorStr = errorStr[:len(errorStr)-1] - s = s[:len(s)-1] - } + fails := s[len(s)-1][0] == '[' + var errorStr string + if fails { + errorStr = s[len(s)-1][1:] + errorStr = errorStr[:len(errorStr)-1] + s = s[:len(s)-1] + } - var ( - g fs.File - err error - ) - switch s[0] { - case "create": - g, err = e.Create(s[1]) - case "create-with-sync": - g, err = e.CreateWithSync(s[1], 1) - case "link": - err = e.Link(s[1], s[2]) - case "open": - g, err = e.Open(s[1]) - case "opendir": - g, err = e.OpenDir(s[1]) - case "delete": - err = e.Remove(s[1]) - case "rename": - err = e.Rename(s[1], s[2]) - case "create-dir": - err = e.MkdirAll(s[1]) - case "list-dir": - result, err := e.List(s[1]) - if err != nil { - break - } - sort.Sort(stringSorter(result)) - got := strings.Join(result, ",") - want := s[3] - if got != want { - t.Fatalf("%q: got %s, want %s", tc, got, want) - } - case "f.write": - _, err = f.Write([]byte(s[1])) - case "f.read": - n, _ := strconv.Atoi(s[1]) - buf := make([]byte, n) - _, err = io.ReadFull(f, buf) - if err != nil { - break - } - if got, want := string(buf), s[3]; got != want { - t.Fatalf("%q: got %q, want %q", tc, got, want) - } - case "f.readat": - n, _ := strconv.Atoi(s[1]) - off, _ := strconv.Atoi(s[2]) - buf := make([]byte, n) - _, err = f.ReadAt(buf, int64(off)) - if err != nil { - break - } - if got, want := string(buf), s[4]; got != want { - t.Fatalf("%q: got %q, want %q", tc, got, want) - } - case "f.close": - f, err = nil, f.Close() - case "f.sync": - err = f.Sync() - default: - t.Fatalf("bad test case: %q", tc) - } + var ( + g fs.File + err error + ) + switch s[0] { + case "create": + g, err = e.Create(s[1]) + case "create-with-sync": + g, err = e.CreateWithSync(s[1], 1) + case "link": + err = e.Link(s[1], s[2]) + case "open": + g, err = e.Open(s[1]) + case "opendir": + g, err = e.OpenDir(s[1]) + case "delete": + err = e.Remove(s[1]) + case "rename": + err = e.Rename(s[1], s[2]) + case "create-dir": + err = e.MkdirAll(s[1]) + case "list-dir": + result, err := e.List(s[1]) + if err != nil { + break + } + sort.Sort(stringSorter(result)) + got := strings.Join(result, ",") + want := s[3] + if got != want { + t.Fatalf("%q: got %s, want %s", tc, got, want) + } + case "f.write": + _, err = f.Write([]byte(s[1])) + case "f.read": + n, _ := strconv.Atoi(s[1]) + buf := make([]byte, n) + _, err = io.ReadFull(f, buf) + if err != nil { + break + } + if got, want := string(buf), s[3]; got != want { + t.Fatalf("%q: got %q, want %q", tc, got, want) + } + case "f.readat": + n, _ := strconv.Atoi(s[1]) + off, _ := strconv.Atoi(s[2]) + buf := make([]byte, n) + _, err = f.ReadAt(buf, int64(off)) + if err != nil { + break + } + if got, want := string(buf), s[4]; got != want { + t.Fatalf("%q: got %q, want %q", tc, got, want) + } + case "f.close": + f, err = nil, f.Close() + case "f.sync": + err = f.Sync() + default: + t.Fatalf("bad test case: %q", tc) + } - if saveF { - f, g = g, nil - } else if g != nil { - g.Close() - } + if saveF { + f, g = g, nil + } else if g != nil { + g.Close() + } - if fails { - if err == nil { - t.Fatalf("%q: got nil error, want non-nil %s", tc, errorStr) - } - var actualErrStr string - if oserror.IsExist(err) { - actualErrStr = "exists" - } else if oserror.IsNotExist(err) { - actualErrStr = "does-not-exist" - } else { - actualErrStr = "error" - } - if errorStr != actualErrStr { - t.Fatalf("%q: got %s, want %s", tc, actualErrStr, errorStr) - } - } else { - if err != nil { - t.Fatalf("%q: %v", tc, err) - } - } + if fails { + if err == nil { + t.Fatalf("%q: got nil error, want non-nil %s", tc, errorStr) } - }) + var actualErrStr string + if oserror.IsExist(err) { + actualErrStr = "exists" + } else if oserror.IsNotExist(err) { + actualErrStr = "does-not-exist" + } else { + actualErrStr = "error" + } + if errorStr != actualErrStr { + t.Fatalf("%q: got %s, want %s", tc, actualErrStr, errorStr) + } + } else { + if err != nil { + t.Fatalf("%q: %v", tc, err) + } + } } } @@ -1358,43 +1297,28 @@ func TestEngineFSFileNotFoundError(t *testing.T) { func TestSupportsPrev(t *testing.T) { defer leaktest.AfterTest(t)() + eng, err := Open(context.Background(), InMemory(), CacheSize(1<<20 /* 1 MiB */)) + require.NoError(t, err) + defer eng.Close() opts := IterOptions{LowerBound: keys.LocalMax, UpperBound: keys.MaxKey} - type engineTest struct { - engineIterSupportsPrev bool - batchIterSupportsPrev bool - snapshotIterSupportsPrev bool - } - runTest := func(t *testing.T, eng Engine, et engineTest) { - t.Run("engine", func(t *testing.T) { - it := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind, opts) - defer it.Close() - require.Equal(t, et.engineIterSupportsPrev, it.SupportsPrev()) - }) - t.Run("batch", func(t *testing.T) { - batch := eng.NewBatch() - defer batch.Close() - batchIt := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, opts) - defer batchIt.Close() - require.Equal(t, et.batchIterSupportsPrev, batchIt.SupportsPrev()) - }) - t.Run("snapshot", func(t *testing.T) { - snap := eng.NewSnapshot() - defer snap.Close() - snapIt := snap.NewMVCCIterator(MVCCKeyAndIntentsIterKind, opts) - defer snapIt.Close() - require.Equal(t, et.snapshotIterSupportsPrev, snapIt.SupportsPrev()) - }) - } - t.Run("pebble", func(t *testing.T) { - - eng, err := Open(context.Background(), InMemory(), CacheSize(1<<20 /* 1 MiB */)) - require.NoError(t, err) - defer eng.Close() - runTest(t, eng, engineTest{ - engineIterSupportsPrev: true, - batchIterSupportsPrev: true, - snapshotIterSupportsPrev: true, - }) + t.Run("engine", func(t *testing.T) { + it := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind, opts) + defer it.Close() + require.Equal(t, true, it.SupportsPrev()) + }) + t.Run("batch", func(t *testing.T) { + batch := eng.NewBatch() + defer batch.Close() + batchIt := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, opts) + defer batchIt.Close() + require.Equal(t, true, batchIt.SupportsPrev()) + }) + t.Run("snapshot", func(t *testing.T) { + snap := eng.NewSnapshot() + defer snap.Close() + snapIt := snap.NewMVCCIterator(MVCCKeyAndIntentsIterKind, opts) + defer snapIt.Close() + require.Equal(t, true, snapIt.SupportsPrev()) }) } diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index 5f920d49d25c..072c7c0418ac 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -444,109 +444,105 @@ func TestMVCCIncrementalIteratorNextIgnoringTime(t *testing.T) { kv2_4_4 := makeKVT(testKey2, testValue4, ts4) kv1_3Deleted := makeKVT(testKey1, roachpb.Value{}, ts3) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + e := NewDefaultInMemForTesting() + defer e.Close() - t.Run("empty", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts3, nil, false) - }) - - for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { - v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { - t.Fatal(err) - } - } + t.Run("empty", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts3, nil, false) + }) - // Exercise time ranges. - t.Run("ts (0-0]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, tsMin, nil, false) - }) - // Returns the kv_2_2_2 even though it is outside (startTime, endTime]. - t.Run("ts (0-1]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts1, kvs(kv1_1_1, kv2_2_2), false) - }) - t.Run("ts (0-∞]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, tsMax, kvs(kv1_2_2, kv1_1_1, - kv2_2_2), false) - }) - t.Run("ts (1-1]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, ts1, nil, false) - }) - // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. - t.Run("ts (1-2]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, ts2, kvs(kv1_2_2, kv1_1_1, - kv2_2_2), false) - }) - t.Run("ts (2-2]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts2, ts2, nil, false) - }) + for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { + v := roachpb.Value{RawBytes: kv.Value} + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + t.Fatal(err) + } + } - // Exercise key ranges. - t.Run("kv [1-1)", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, testKey1, testKey1, tsMin, tsMax, nil, false) - }) - t.Run("kv [1-2)", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, testKey1, testKey2, tsMin, tsMax, kvs(kv1_2_2, - kv1_1_1), false) - }) + // Exercise time ranges. + t.Run("ts (0-0]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, tsMin, nil, false) + }) + // Returns the kv_2_2_2 even though it is outside (startTime, endTime]. + t.Run("ts (0-1]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts1, kvs(kv1_1_1, kv2_2_2), false) + }) + t.Run("ts (0-∞]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, tsMax, kvs(kv1_2_2, kv1_1_1, + kv2_2_2), false) + }) + t.Run("ts (1-1]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, ts1, nil, false) + }) + // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. + t.Run("ts (1-2]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, ts2, kvs(kv1_2_2, kv1_1_1, + kv2_2_2), false) + }) + t.Run("ts (2-2]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts2, ts2, nil, false) + }) - // Exercise deletion. - if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } - // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. - t.Run("del", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, tsMax, kvs(kv1_3Deleted, - kv1_2_2, kv1_1_1, kv2_2_2), false) - }) + // Exercise key ranges. + t.Run("kv [1-1)", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, testKey1, testKey1, tsMin, tsMax, nil, false) + }) + t.Run("kv [1-2)", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, testKey1, testKey2, tsMin, tsMax, kvs(kv1_2_2, + kv1_1_1), false) + }) - // Insert an intent of testKey2. - txn1ID := uuid.MakeV4() - txn1 := roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ - Key: testKey2, - ID: txn1ID, - Epoch: 1, - WriteTimestamp: ts4, - }, - ReadTimestamp: ts4, - } - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { - t.Fatal(err) - } + // Exercise deletion. + if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } + // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. + t.Run("del", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, tsMax, kvs(kv1_3Deleted, + kv1_2_2, kv1_1_1, kv2_2_2), false) + }) - // We have to be careful that we are testing the intent handling logic of - // NextIgnoreTime() rather than the first SeekGE(). We do this by - // ensuring that the SeekGE() doesn't encounter an intent. - t.Run("intents", func(t *testing.T) { - ignoreTimeExpectErr(t, e, testKey1, testKey2.PrefixEnd(), tsMin, tsMax, - "conflicting intents", false) - }) - t.Run("intents", func(t *testing.T) { - ignoreTimeExpectErr(t, e, localMax, keyMax, tsMin, ts4, "conflicting intents", false) - }) - // Intents above the upper time bound or beneath the lower time bound must - // be ignored. Note that the lower time bound is exclusive while the upper - // time bound is inclusive. - // - // The intent at ts=4 for kv2 lies outside the timespan - // (startTime, endTime] so we do not raise an error and just move on to - // its versioned KV, a provisional value. - t.Run("intents", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts3, kvs(kv1_3Deleted, - kv1_2_2, kv1_1_1, kv2_4_4, kv2_2_2), false) - }) - t.Run("intents", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts4, tsMax, kvs(), false) - }) - t.Run("intents", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts4.Next(), tsMax, kvs(), false) - }) - }) + // Insert an intent of testKey2. + txn1ID := uuid.MakeV4() + txn1 := roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ + Key: testKey2, + ID: txn1ID, + Epoch: 1, + WriteTimestamp: ts4, + }, + ReadTimestamp: ts4, + } + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { + t.Fatal(err) } + + // We have to be careful that we are testing the intent handling logic of + // NextIgnoreTime() rather than the first SeekGE(). We do this by + // ensuring that the SeekGE() doesn't encounter an intent. + t.Run("intents", func(t *testing.T) { + ignoreTimeExpectErr(t, e, testKey1, testKey2.PrefixEnd(), tsMin, tsMax, + "conflicting intents", false) + }) + t.Run("intents", func(t *testing.T) { + ignoreTimeExpectErr(t, e, localMax, keyMax, tsMin, ts4, "conflicting intents", false) + }) + // Intents above the upper time bound or beneath the lower time bound must + // be ignored. Note that the lower time bound is exclusive while the upper + // time bound is inclusive. + // + // The intent at ts=4 for kv2 lies outside the timespan + // (startTime, endTime] so we do not raise an error and just move on to + // its versioned KV, a provisional value. + t.Run("intents", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts3, kvs(kv1_3Deleted, + kv1_2_2, kv1_1_1, kv2_4_4, kv2_2_2), false) + }) + t.Run("intents", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts4, tsMax, kvs(), false) + }) + t.Run("intents", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts4.Next(), tsMax, kvs(), false) + }) } // TestMVCCIncrementalIteratorNextKeyIgnoringTime tests the iteration semantics @@ -584,106 +580,102 @@ func TestMVCCIncrementalIteratorNextKeyIgnoringTime(t *testing.T) { kv2_4_4 := makeKVT(testKey2, testValue4, ts4) kv1_3Deleted := makeKVT(testKey1, roachpb.Value{}, ts3) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() - - t.Run("empty", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts3, nil, true) - }) + e := NewDefaultInMemForTesting() + defer e.Close() - for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { - v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { - t.Fatal(err) - } - } + t.Run("empty", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts3, nil, true) + }) - // Exercise time ranges. - t.Run("ts (0-0]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, tsMin, nil, true) - }) - // Returns the kv_2_2_2 even though it is outside (startTime, endTime]. - t.Run("ts (0-1]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts1, kvs(kv1_1_1, kv2_2_2), true) - }) - t.Run("ts (0-∞]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, tsMax, kvs(kv1_2_2, kv2_2_2), - true) - }) - t.Run("ts (1-1]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, ts1, nil, true) - }) - t.Run("ts (1-2]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, ts2, kvs(kv1_2_2, kv2_2_2), true) - }) - t.Run("ts (2-2]", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts2, ts2, nil, true) - }) + for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { + v := roachpb.Value{RawBytes: kv.Value} + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + t.Fatal(err) + } + } - // Exercise key ranges. - t.Run("kv [1-1)", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, testKey1, testKey1, tsMin, tsMax, nil, true) - }) - t.Run("kv [1-2)", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, testKey1, testKey2, tsMin, tsMax, kvs(kv1_2_2), true) - }) + // Exercise time ranges. + t.Run("ts (0-0]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, tsMin, nil, true) + }) + // Returns the kv_2_2_2 even though it is outside (startTime, endTime]. + t.Run("ts (0-1]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts1, kvs(kv1_1_1, kv2_2_2), true) + }) + t.Run("ts (0-∞]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, tsMax, kvs(kv1_2_2, kv2_2_2), + true) + }) + t.Run("ts (1-1]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, ts1, nil, true) + }) + t.Run("ts (1-2]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, ts2, kvs(kv1_2_2, kv2_2_2), true) + }) + t.Run("ts (2-2]", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts2, ts2, nil, true) + }) - // Exercise deletion. - if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } - // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. - t.Run("del", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, tsMax, kvs(kv1_3Deleted, - kv2_2_2), true) - }) + // Exercise key ranges. + t.Run("kv [1-1)", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, testKey1, testKey1, tsMin, tsMax, nil, true) + }) + t.Run("kv [1-2)", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, testKey1, testKey2, tsMin, tsMax, kvs(kv1_2_2), true) + }) - // Insert an intent of testKey2. - txn1ID := uuid.MakeV4() - txn1 := roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ - Key: testKey2, - ID: txn1ID, - Epoch: 1, - WriteTimestamp: ts4, - }, - ReadTimestamp: ts4, - } - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { - t.Fatal(err) - } + // Exercise deletion. + if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } + // Returns the kv_1_1_1 even though it is outside (startTime, endTime]. + t.Run("del", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts1, tsMax, kvs(kv1_3Deleted, + kv2_2_2), true) + }) - // We have to be careful that we are testing the intent handling logic of - // NextIgnoreTime() rather than the first SeekGE(). We do this by - // ensuring that the SeekGE() doesn't encounter an intent. - t.Run("intents", func(t *testing.T) { - ignoreTimeExpectErr(t, e, testKey1, testKey2.PrefixEnd(), tsMin, tsMax, - "conflicting intents", true) - }) - t.Run("intents", func(t *testing.T) { - ignoreTimeExpectErr(t, e, localMax, keyMax, tsMin, ts4, "conflicting intents", true) - }) - // Intents above the upper time bound or beneath the lower time bound must - // be ignored. Note that the lower time bound is exclusive while the upper - // time bound is inclusive. - // - // The intent at ts=4 for kv2 lies outside the timespan - // (startTime, endTime] so we do not raise an error and just move on to - // its versioned KV, a provisional value. - t.Run("intents", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts3, kvs(kv1_3Deleted, - kv2_4_4), true) - }) - t.Run("intents", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts4, tsMax, kvs(), true) - }) - t.Run("intents", func(t *testing.T) { - assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts4.Next(), tsMax, kvs(), true) - }) - }) + // Insert an intent of testKey2. + txn1ID := uuid.MakeV4() + txn1 := roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ + Key: testKey2, + ID: txn1ID, + Epoch: 1, + WriteTimestamp: ts4, + }, + ReadTimestamp: ts4, + } + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { + t.Fatal(err) } + + // We have to be careful that we are testing the intent handling logic of + // NextIgnoreTime() rather than the first SeekGE(). We do this by + // ensuring that the SeekGE() doesn't encounter an intent. + t.Run("intents", func(t *testing.T) { + ignoreTimeExpectErr(t, e, testKey1, testKey2.PrefixEnd(), tsMin, tsMax, + "conflicting intents", true) + }) + t.Run("intents", func(t *testing.T) { + ignoreTimeExpectErr(t, e, localMax, keyMax, tsMin, ts4, "conflicting intents", true) + }) + // Intents above the upper time bound or beneath the lower time bound must + // be ignored. Note that the lower time bound is exclusive while the upper + // time bound is inclusive. + // + // The intent at ts=4 for kv2 lies outside the timespan + // (startTime, endTime] so we do not raise an error and just move on to + // its versioned KV, a provisional value. + t.Run("intents", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, tsMin, ts3, kvs(kv1_3Deleted, + kv2_4_4), true) + }) + t.Run("intents", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts4, tsMax, kvs(), true) + }) + t.Run("intents", func(t *testing.T) { + assertIgnoreTimeIteratedKVs(t, e, localMax, keyMax, ts4.Next(), tsMax, kvs(), true) + }) } func TestMVCCIncrementalIteratorInlinePolicy(t *testing.T) { @@ -714,45 +706,41 @@ func TestMVCCIncrementalIteratorInlinePolicy(t *testing.T) { kv2_2_2 := makeKVT(testKey2, testValue2, ts2) inline3_2_1 := makeKVT(testKey3, testValue2, hlc.Timestamp{}) - for _, engineImpl := range mvccEngineImpls { - e := engineImpl.create() - defer e.Close() - for _, kv := range []MVCCKeyValue{inline1_1_1, kv2_1_1, kv2_2_2, inline3_2_1} { - v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { - t.Fatal(err) - } + e := NewDefaultInMemForTesting() + defer e.Close() + for _, kv := range []MVCCKeyValue{inline1_1_1, kv2_1_1, kv2_2_2, inline3_2_1} { + v := roachpb.Value{RawBytes: kv.Value} + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + t.Fatal(err) } - t.Run(engineImpl.name, func(t *testing.T) { - t.Run("returns error if inline value is found", func(t *testing.T) { - iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ - EndKey: keyMax, - StartTime: tsMin, - EndTime: tsMax, - }) - defer iter.Close() - iter.SeekGE(MakeMVCCMetadataKey(testKey1)) - _, err := iter.Valid() - assert.EqualError(t, err, "unexpected inline value found: \"/db1\"") + } + t.Run("returns error if inline value is found", func(t *testing.T) { + iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ + EndKey: keyMax, + StartTime: tsMin, + EndTime: tsMax, + }) + defer iter.Close() + iter.SeekGE(MakeMVCCMetadataKey(testKey1)) + _, err := iter.Valid() + assert.EqualError(t, err, "unexpected inline value found: \"/db1\"") + }) + t.Run("returns error on NextIgnoringTime if inline value is found", + func(t *testing.T) { + iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ + EndKey: keyMax, + StartTime: tsMin, + EndTime: tsMax, }) - t.Run("returns error on NextIgnoringTime if inline value is found", - func(t *testing.T) { - iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ - EndKey: keyMax, - StartTime: tsMin, - EndTime: tsMax, - }) - defer iter.Close() - iter.SeekGE(MakeMVCCMetadataKey(testKey2)) - expectKeyValue(t, iter, kv2_2_2) - iter.NextIgnoringTime() - expectKeyValue(t, iter, kv2_1_1) - iter.NextIgnoringTime() - _, err := iter.Valid() - assert.EqualError(t, err, "unexpected inline value found: \"/db3\"") - }) + defer iter.Close() + iter.SeekGE(MakeMVCCMetadataKey(testKey2)) + expectKeyValue(t, iter, kv2_2_2) + iter.NextIgnoringTime() + expectKeyValue(t, iter, kv2_1_1) + iter.NextIgnoringTime() + _, err := iter.Valid() + assert.EqualError(t, err, "unexpected inline value found: \"/db3\"") }) - } } func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { @@ -788,101 +776,97 @@ func TestMVCCIncrementalIteratorIntentPolicy(t *testing.T) { intentErr := &roachpb.WriteIntentError{Intents: []roachpb.Intent{intent2_2_2}} - for _, engineImpl := range mvccEngineImpls { - e := engineImpl.create() - defer e.Close() - for _, kv := range []MVCCKeyValue{kv1_1_1, kv1_2_2, kv1_3_3, kv2_1_1} { - v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { - t.Fatal(err) - } - } - if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, testValue2, &txn); err != nil { + e := NewDefaultInMemForTesting() + defer e.Close() + for _, kv := range []MVCCKeyValue{kv1_1_1, kv1_2_2, kv1_3_3, kv2_1_1} { + v := roachpb.Value{RawBytes: kv.Value} + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { t.Fatal(err) } - t.Run(engineImpl.name, func(t *testing.T) { - t.Run("PolicyError returns error if an intent is in the time range", func(t *testing.T) { - iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ - EndKey: keyMax, - StartTime: tsMin, - EndTime: tsMax, - IntentPolicy: MVCCIncrementalIterIntentPolicyError, - }) - defer iter.Close() - iter.SeekGE(MakeMVCCMetadataKey(testKey1)) - for ; ; iter.Next() { - if ok, _ := iter.Valid(); !ok || iter.UnsafeKey().Key.Compare(keyMax) >= 0 { - break - } - } - _, err := iter.Valid() - assert.EqualError(t, err, intentErr.Error()) + } + if err := MVCCPut(ctx, e, nil, txn.TxnMeta.Key, txn.ReadTimestamp, hlc.ClockTimestamp{}, testValue2, &txn); err != nil { + t.Fatal(err) + } + t.Run("PolicyError returns error if an intent is in the time range", func(t *testing.T) { + iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ + EndKey: keyMax, + StartTime: tsMin, + EndTime: tsMax, + IntentPolicy: MVCCIncrementalIterIntentPolicyError, + }) + defer iter.Close() + iter.SeekGE(MakeMVCCMetadataKey(testKey1)) + for ; ; iter.Next() { + if ok, _ := iter.Valid(); !ok || iter.UnsafeKey().Key.Compare(keyMax) >= 0 { + break + } + } + _, err := iter.Valid() + assert.EqualError(t, err, intentErr.Error()) - iter.SeekGE(MakeMVCCMetadataKey(testKey1)) - _, err = iter.Valid() - require.NoError(t, err) - for ; ; iter.NextIgnoringTime() { - if ok, err := iter.Valid(); !ok { - assert.EqualError(t, err, intentErr.Error()) - break - } - } - }) - t.Run("PolicyError ignores intents outside of time range", func(t *testing.T) { - iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ - EndKey: keyMax, - StartTime: ts2, - EndTime: tsMax, - IntentPolicy: MVCCIncrementalIterIntentPolicyError, - }) - defer iter.Close() - iter.SeekGE(MakeMVCCMetadataKey(testKey1)) - expectKeyValue(t, iter, kv1_3_3) - iter.Next() - valid, err := iter.Valid() - assert.NoError(t, err) - assert.False(t, valid) - }) - t.Run("PolicyEmit returns inline values to caller", func(t *testing.T) { - iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ - EndKey: keyMax, - StartTime: tsMin, - EndTime: tsMax, - IntentPolicy: MVCCIncrementalIterIntentPolicyEmit, - }) - defer iter.Close() - testIterWithNextFunc := func(nextFunc func()) { - iter.SeekGE(MakeMVCCMetadataKey(testKey1)) - for _, kv := range []MVCCKeyValue{kv1_3_3, kv1_2_2, kv1_1_1} { - expectKeyValue(t, iter, kv) - nextFunc() - } - expectIntent(t, iter, intent2_2_2) - nextFunc() - expectKeyValue(t, iter, kv2_2_2) - nextFunc() - expectKeyValue(t, iter, kv2_1_1) - } - testIterWithNextFunc(iter.Next) - testIterWithNextFunc(iter.NextIgnoringTime) - }) - t.Run("PolicyEmit ignores intents outside of time range", func(t *testing.T) { - iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ - EndKey: keyMax, - StartTime: ts2, - EndTime: tsMax, - IntentPolicy: MVCCIncrementalIterIntentPolicyEmit, - }) - defer iter.Close() - iter.SeekGE(MakeMVCCMetadataKey(testKey1)) - expectKeyValue(t, iter, kv1_3_3) - iter.Next() - valid, err := iter.Valid() - assert.NoError(t, err) - assert.False(t, valid) - }) + iter.SeekGE(MakeMVCCMetadataKey(testKey1)) + _, err = iter.Valid() + require.NoError(t, err) + for ; ; iter.NextIgnoringTime() { + if ok, err := iter.Valid(); !ok { + assert.EqualError(t, err, intentErr.Error()) + break + } + } + }) + t.Run("PolicyError ignores intents outside of time range", func(t *testing.T) { + iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ + EndKey: keyMax, + StartTime: ts2, + EndTime: tsMax, + IntentPolicy: MVCCIncrementalIterIntentPolicyError, }) - } + defer iter.Close() + iter.SeekGE(MakeMVCCMetadataKey(testKey1)) + expectKeyValue(t, iter, kv1_3_3) + iter.Next() + valid, err := iter.Valid() + assert.NoError(t, err) + assert.False(t, valid) + }) + t.Run("PolicyEmit returns inline values to caller", func(t *testing.T) { + iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ + EndKey: keyMax, + StartTime: tsMin, + EndTime: tsMax, + IntentPolicy: MVCCIncrementalIterIntentPolicyEmit, + }) + defer iter.Close() + testIterWithNextFunc := func(nextFunc func()) { + iter.SeekGE(MakeMVCCMetadataKey(testKey1)) + for _, kv := range []MVCCKeyValue{kv1_3_3, kv1_2_2, kv1_1_1} { + expectKeyValue(t, iter, kv) + nextFunc() + } + expectIntent(t, iter, intent2_2_2) + nextFunc() + expectKeyValue(t, iter, kv2_2_2) + nextFunc() + expectKeyValue(t, iter, kv2_1_1) + } + testIterWithNextFunc(iter.Next) + testIterWithNextFunc(iter.NextIgnoringTime) + }) + t.Run("PolicyEmit ignores intents outside of time range", func(t *testing.T) { + iter := NewMVCCIncrementalIterator(e, MVCCIncrementalIterOptions{ + EndKey: keyMax, + StartTime: ts2, + EndTime: tsMax, + IntentPolicy: MVCCIncrementalIterIntentPolicyEmit, + }) + defer iter.Close() + iter.SeekGE(MakeMVCCMetadataKey(testKey1)) + expectKeyValue(t, iter, kv1_3_3) + iter.Next() + valid, err := iter.Valid() + assert.NoError(t, err) + assert.False(t, valid) + }) } func expectKeyValue(t *testing.T, iter SimpleMVCCIterator, kv MVCCKeyValue) { @@ -951,142 +935,138 @@ func TestMVCCIncrementalIterator(t *testing.T) { kv2_2_2 := makeKVT(testKey2, testValue3, ts2) kv1Deleted3 := makeKVT(testKey1, roachpb.Value{}, ts3) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name+"-latest", func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + t.Run("latest", func(t *testing.T) { + e := NewDefaultInMemForTesting() + defer e.Close() - t.Run("empty", assertEqualKVs(e, localMax, keyMax, tsMin, ts3, latest, nil)) + t.Run("empty", assertEqualKVs(e, localMax, keyMax, tsMin, ts3, latest, nil)) - for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { - v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { - t.Fatal(err) - } + for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { + v := roachpb.Value{RawBytes: kv.Value} + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + t.Fatal(err) } + } - // Exercise time ranges. - t.Run("ts (0-0]", assertEqualKVs(e, localMax, keyMax, tsMin, tsMin, latest, nil)) - t.Run("ts (0-1]", assertEqualKVs(e, localMax, keyMax, tsMin, ts1, latest, kvs(kv1_1_1))) - t.Run("ts (0-∞]", assertEqualKVs(e, localMax, keyMax, tsMin, tsMax, latest, kvs(kv1_2_2, kv2_2_2))) - t.Run("ts (1-1]", assertEqualKVs(e, localMax, keyMax, ts1, ts1, latest, nil)) - t.Run("ts (1-2]", assertEqualKVs(e, localMax, keyMax, ts1, ts2, latest, kvs(kv1_2_2, kv2_2_2))) - t.Run("ts (2-2]", assertEqualKVs(e, localMax, keyMax, ts2, ts2, latest, nil)) + // Exercise time ranges. + t.Run("ts (0-0]", assertEqualKVs(e, localMax, keyMax, tsMin, tsMin, latest, nil)) + t.Run("ts (0-1]", assertEqualKVs(e, localMax, keyMax, tsMin, ts1, latest, kvs(kv1_1_1))) + t.Run("ts (0-∞]", assertEqualKVs(e, localMax, keyMax, tsMin, tsMax, latest, kvs(kv1_2_2, kv2_2_2))) + t.Run("ts (1-1]", assertEqualKVs(e, localMax, keyMax, ts1, ts1, latest, nil)) + t.Run("ts (1-2]", assertEqualKVs(e, localMax, keyMax, ts1, ts2, latest, kvs(kv1_2_2, kv2_2_2))) + t.Run("ts (2-2]", assertEqualKVs(e, localMax, keyMax, ts2, ts2, latest, nil)) - // Exercise key ranges. - t.Run("kv [1-1)", assertEqualKVs(e, testKey1, testKey1, tsMin, tsMax, latest, nil)) - t.Run("kv [1-2)", assertEqualKVs(e, testKey1, testKey2, tsMin, tsMax, latest, kvs(kv1_2_2))) + // Exercise key ranges. + t.Run("kv [1-1)", assertEqualKVs(e, testKey1, testKey1, tsMin, tsMax, latest, nil)) + t.Run("kv [1-2)", assertEqualKVs(e, testKey1, testKey2, tsMin, tsMax, latest, kvs(kv1_2_2))) - // Exercise deletion. - if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } - t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, latest, kvs(kv1Deleted3, kv2_2_2))) + // Exercise deletion. + if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } + t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, latest, kvs(kv1Deleted3, kv2_2_2))) - // Exercise intent handling. - txn1, intentErr1 := makeKVTxn(testKey1, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { - t.Fatal(err) - } - txn2, intentErr2 := makeKVTxn(testKey2, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn2); err != nil { - t.Fatal(err) - } - t.Run("intents-1", - iterateExpectErr(e, testKey1, testKey1.PrefixEnd(), tsMin, tsMax, latest, intents(intentErr1))) - t.Run("intents-2", - iterateExpectErr(e, testKey2, testKey2.PrefixEnd(), tsMin, tsMax, latest, intents(intentErr2))) - t.Run("intents-multi", - iterateExpectErr(e, localMax, keyMax, tsMin, ts4, latest, intents(intentErr1, intentErr2))) - // Intents above the upper time bound or beneath the lower time bound must - // be ignored (#28358). Note that the lower time bound is exclusive while - // the upper time bound is inclusive. - t.Run("intents-filtered-1", assertEqualKVs(e, localMax, keyMax, tsMin, ts3, latest, kvs(kv1Deleted3, kv2_2_2))) - t.Run("intents-filtered-2", assertEqualKVs(e, localMax, keyMax, ts4, tsMax, latest, kvs())) - t.Run("intents-filtered-3", assertEqualKVs(e, localMax, keyMax, ts4.Next(), tsMax, latest, kvs())) - - intent1 := roachpb.MakeLockUpdate(&txn1, roachpb.Span{Key: testKey1}) - intent1.Status = roachpb.COMMITTED - if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent1); err != nil { - t.Fatal(err) - } - intent2 := roachpb.MakeLockUpdate(&txn2, roachpb.Span{Key: testKey2}) - intent2.Status = roachpb.ABORTED - if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent2); err != nil { - t.Fatal(err) - } - t.Run("intents-resolved", assertEqualKVs(e, localMax, keyMax, tsMin, tsMax, latest, kvs(kv1_4_4, kv2_2_2))) - }) - } + // Exercise intent handling. + txn1, intentErr1 := makeKVTxn(testKey1, ts4) + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { + t.Fatal(err) + } + txn2, intentErr2 := makeKVTxn(testKey2, ts4) + if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn2); err != nil { + t.Fatal(err) + } + t.Run("intents-1", + iterateExpectErr(e, testKey1, testKey1.PrefixEnd(), tsMin, tsMax, latest, intents(intentErr1))) + t.Run("intents-2", + iterateExpectErr(e, testKey2, testKey2.PrefixEnd(), tsMin, tsMax, latest, intents(intentErr2))) + t.Run("intents-multi", + iterateExpectErr(e, localMax, keyMax, tsMin, ts4, latest, intents(intentErr1, intentErr2))) + // Intents above the upper time bound or beneath the lower time bound must + // be ignored (#28358). Note that the lower time bound is exclusive while + // the upper time bound is inclusive. + t.Run("intents-filtered-1", assertEqualKVs(e, localMax, keyMax, tsMin, ts3, latest, kvs(kv1Deleted3, kv2_2_2))) + t.Run("intents-filtered-2", assertEqualKVs(e, localMax, keyMax, ts4, tsMax, latest, kvs())) + t.Run("intents-filtered-3", assertEqualKVs(e, localMax, keyMax, ts4.Next(), tsMax, latest, kvs())) + + intent1 := roachpb.MakeLockUpdate(&txn1, roachpb.Span{Key: testKey1}) + intent1.Status = roachpb.COMMITTED + if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent1); err != nil { + t.Fatal(err) + } + intent2 := roachpb.MakeLockUpdate(&txn2, roachpb.Span{Key: testKey2}) + intent2.Status = roachpb.ABORTED + if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent2); err != nil { + t.Fatal(err) + } + t.Run("intents-resolved", assertEqualKVs(e, localMax, keyMax, tsMin, tsMax, latest, kvs(kv1_4_4, kv2_2_2))) + }) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name+"-all", func(t *testing.T) { - e := engineImpl.create() - defer e.Close() + t.Run("all", func(t *testing.T) { + e := NewDefaultInMemForTesting() + defer e.Close() - t.Run("empty", assertEqualKVs(e, localMax, keyMax, tsMin, ts3, all, nil)) + t.Run("empty", assertEqualKVs(e, localMax, keyMax, tsMin, ts3, all, nil)) - for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { - v := roachpb.Value{RawBytes: kv.Value} - if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { - t.Fatal(err) - } + for _, kv := range kvs(kv1_1_1, kv1_2_2, kv2_2_2) { + v := roachpb.Value{RawBytes: kv.Value} + if err := MVCCPut(ctx, e, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v, nil); err != nil { + t.Fatal(err) } + } - // Exercise time ranges. - t.Run("ts (0-0]", assertEqualKVs(e, localMax, keyMax, tsMin, tsMin, all, nil)) - t.Run("ts (0-1]", assertEqualKVs(e, localMax, keyMax, tsMin, ts1, all, kvs(kv1_1_1))) - t.Run("ts (0-∞]", assertEqualKVs(e, localMax, keyMax, tsMin, tsMax, all, kvs(kv1_2_2, kv1_1_1, kv2_2_2))) - t.Run("ts (1-1]", assertEqualKVs(e, localMax, keyMax, ts1, ts1, all, nil)) - t.Run("ts (1-2]", assertEqualKVs(e, localMax, keyMax, ts1, ts2, all, kvs(kv1_2_2, kv2_2_2))) - t.Run("ts (2-2]", assertEqualKVs(e, localMax, keyMax, ts2, ts2, all, nil)) + // Exercise time ranges. + t.Run("ts (0-0]", assertEqualKVs(e, localMax, keyMax, tsMin, tsMin, all, nil)) + t.Run("ts (0-1]", assertEqualKVs(e, localMax, keyMax, tsMin, ts1, all, kvs(kv1_1_1))) + t.Run("ts (0-∞]", assertEqualKVs(e, localMax, keyMax, tsMin, tsMax, all, kvs(kv1_2_2, kv1_1_1, kv2_2_2))) + t.Run("ts (1-1]", assertEqualKVs(e, localMax, keyMax, ts1, ts1, all, nil)) + t.Run("ts (1-2]", assertEqualKVs(e, localMax, keyMax, ts1, ts2, all, kvs(kv1_2_2, kv2_2_2))) + t.Run("ts (2-2]", assertEqualKVs(e, localMax, keyMax, ts2, ts2, all, nil)) - // Exercise key ranges. - t.Run("kv [1-1)", assertEqualKVs(e, testKey1, testKey1, tsMin, tsMax, all, nil)) - t.Run("kv [1-2)", assertEqualKVs(e, testKey1, testKey2, tsMin, tsMax, all, kvs(kv1_2_2, kv1_1_1))) + // Exercise key ranges. + t.Run("kv [1-1)", assertEqualKVs(e, testKey1, testKey1, tsMin, tsMax, all, nil)) + t.Run("kv [1-2)", assertEqualKVs(e, testKey1, testKey2, tsMin, tsMax, all, kvs(kv1_2_2, kv1_1_1))) - // Exercise deletion. - if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } - t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, all, kvs(kv1Deleted3, kv1_2_2, kv2_2_2))) + // Exercise deletion. + if _, err := MVCCDelete(ctx, e, nil, testKey1, ts3, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } + t.Run("del", assertEqualKVs(e, localMax, keyMax, ts1, tsMax, all, kvs(kv1Deleted3, kv1_2_2, kv2_2_2))) - // Exercise intent handling. - txn1, intentErr1 := makeKVTxn(testKey1, ts4) - if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { - t.Fatal(err) - } - txn2, intentErr2 := makeKVTxn(testKey2, ts4) - if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn2); err != nil { - t.Fatal(err) - } - // Single intent tests are verifying behavior when intent collection is not enabled. - t.Run("intents-1", - iterateExpectErr(e, testKey1, testKey1.PrefixEnd(), tsMin, tsMax, all, intents(intentErr1))) - t.Run("intents-2", - iterateExpectErr(e, testKey2, testKey2.PrefixEnd(), tsMin, tsMax, all, intents(intentErr2))) - t.Run("intents-multi", - iterateExpectErr(e, localMax, keyMax, tsMin, ts4, all, intents(intentErr1, intentErr2))) - // Intents above the upper time bound or beneath the lower time bound must - // be ignored (#28358). Note that the lower time bound is exclusive while - // the upper time bound is inclusive. - t.Run("intents-filtered-1", assertEqualKVs(e, localMax, keyMax, tsMin, ts3, all, kvs(kv1Deleted3, kv1_2_2, kv1_1_1, kv2_2_2))) - t.Run("intents-filtered-2", assertEqualKVs(e, localMax, keyMax, ts4, tsMax, all, kvs())) - t.Run("intents-filtered-3", assertEqualKVs(e, localMax, keyMax, ts4.Next(), tsMax, all, kvs())) - - intent1 := roachpb.MakeLockUpdate(&txn1, roachpb.Span{Key: testKey1}) - intent1.Status = roachpb.COMMITTED - if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent1); err != nil { - t.Fatal(err) - } - intent2 := roachpb.MakeLockUpdate(&txn2, roachpb.Span{Key: testKey2}) - intent2.Status = roachpb.ABORTED - if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent2); err != nil { - t.Fatal(err) - } - t.Run("intents-resolved", assertEqualKVs(e, localMax, keyMax, tsMin, tsMax, all, kvs(kv1_4_4, kv1Deleted3, kv1_2_2, kv1_1_1, kv2_2_2))) - }) - } + // Exercise intent handling. + txn1, intentErr1 := makeKVTxn(testKey1, ts4) + if err := MVCCPut(ctx, e, nil, txn1.TxnMeta.Key, txn1.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn1); err != nil { + t.Fatal(err) + } + txn2, intentErr2 := makeKVTxn(testKey2, ts4) + if err := MVCCPut(ctx, e, nil, txn2.TxnMeta.Key, txn2.ReadTimestamp, hlc.ClockTimestamp{}, testValue4, &txn2); err != nil { + t.Fatal(err) + } + // Single intent tests are verifying behavior when intent collection is not enabled. + t.Run("intents-1", + iterateExpectErr(e, testKey1, testKey1.PrefixEnd(), tsMin, tsMax, all, intents(intentErr1))) + t.Run("intents-2", + iterateExpectErr(e, testKey2, testKey2.PrefixEnd(), tsMin, tsMax, all, intents(intentErr2))) + t.Run("intents-multi", + iterateExpectErr(e, localMax, keyMax, tsMin, ts4, all, intents(intentErr1, intentErr2))) + // Intents above the upper time bound or beneath the lower time bound must + // be ignored (#28358). Note that the lower time bound is exclusive while + // the upper time bound is inclusive. + t.Run("intents-filtered-1", assertEqualKVs(e, localMax, keyMax, tsMin, ts3, all, kvs(kv1Deleted3, kv1_2_2, kv1_1_1, kv2_2_2))) + t.Run("intents-filtered-2", assertEqualKVs(e, localMax, keyMax, ts4, tsMax, all, kvs())) + t.Run("intents-filtered-3", assertEqualKVs(e, localMax, keyMax, ts4.Next(), tsMax, all, kvs())) + + intent1 := roachpb.MakeLockUpdate(&txn1, roachpb.Span{Key: testKey1}) + intent1.Status = roachpb.COMMITTED + if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent1); err != nil { + t.Fatal(err) + } + intent2 := roachpb.MakeLockUpdate(&txn2, roachpb.Span{Key: testKey2}) + intent2.Status = roachpb.ABORTED + if _, err := MVCCResolveWriteIntent(ctx, e, nil, intent2); err != nil { + t.Fatal(err) + } + t.Run("intents-resolved", assertEqualKVs(e, localMax, keyMax, tsMin, tsMax, all, kvs(kv1_4_4, kv1Deleted3, kv1_2_2, kv1_1_1, kv2_2_2))) + }) } func slurpKVsInTimeRange( @@ -1120,80 +1100,76 @@ func TestMVCCIncrementalIteratorIntentRewrittenConcurrently(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - e := engineImpl.create() - defer e.Close() - - // Create a DB containing a single intent. - ctx := context.Background() - - kA := roachpb.Key("kA") - vA1 := roachpb.MakeValueFromString("vA1") - vA2 := roachpb.MakeValueFromString("vA2") - ts0 := hlc.Timestamp{WallTime: 0} - ts1 := hlc.Timestamp{WallTime: 1} - ts2 := hlc.Timestamp{WallTime: 2} - ts3 := hlc.Timestamp{WallTime: 3} - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ - Key: roachpb.Key("b"), - ID: uuid.MakeV4(), - Epoch: 1, - WriteTimestamp: ts1, - Sequence: 1, - }, - ReadTimestamp: ts1, - } - if err := MVCCPut(ctx, e, nil, kA, ts1, hlc.ClockTimestamp{}, vA1, txn); err != nil { - t.Fatal(err) - } + e := NewDefaultInMemForTesting() + defer e.Close() - // Concurrently iterate over the intent using a time-bound iterator and move - // the intent out of the time-bound iterator's time range by writing to it - // again at a higher timestamp. - g, _ := errgroup.WithContext(ctx) - g.Go(func() error { - // Re-write the intent with a higher timestamp. - txn.WriteTimestamp = ts3 - txn.Sequence = 2 - // Use a batch since MVCCPut is not atomic when using an Engine and we - // are not using latches to prevent a concurrent read in the other - // goroutine. A non-atomic Put can cause the strict invariant checking - // in intentInterleavingIter to be violated. - b := e.NewBatch() - defer b.Close() - if err := MVCCPut(ctx, b, nil, kA, ts1, hlc.ClockTimestamp{}, vA2, txn); err != nil { - return err - } - return b.Commit(false) - }) - g.Go(func() error { - // Iterate with a time range that includes the initial intent but does - // not include the new intent. - kvs, err := slurpKVsInTimeRange(e, kA, ts0, ts2) - - // There are two permissible outcomes from the scan. If the iteration - // wins the race with the put that moves the intent then it should - // observe the intent and return a write intent error. If the iteration - // loses the race with the put that moves the intent then it should - // observe and return nothing because there will be no committed or - // provisional keys in its time range. - if err != nil { - if !testutils.IsError(err, `conflicting intents on "kA"`) { - return err - } - } else { - if len(kvs) != 0 { - return errors.Errorf(`unexpected kvs: %v`, kvs) - } - } - return nil - }) - if err := g.Wait(); err != nil { - t.Fatal(err) + // Create a DB containing a single intent. + ctx := context.Background() + + kA := roachpb.Key("kA") + vA1 := roachpb.MakeValueFromString("vA1") + vA2 := roachpb.MakeValueFromString("vA2") + ts0 := hlc.Timestamp{WallTime: 0} + ts1 := hlc.Timestamp{WallTime: 1} + ts2 := hlc.Timestamp{WallTime: 2} + ts3 := hlc.Timestamp{WallTime: 3} + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ + Key: roachpb.Key("b"), + ID: uuid.MakeV4(), + Epoch: 1, + WriteTimestamp: ts1, + Sequence: 1, + }, + ReadTimestamp: ts1, + } + if err := MVCCPut(ctx, e, nil, kA, ts1, hlc.ClockTimestamp{}, vA1, txn); err != nil { + t.Fatal(err) + } + + // Concurrently iterate over the intent using a time-bound iterator and move + // the intent out of the time-bound iterator's time range by writing to it + // again at a higher timestamp. + g, _ := errgroup.WithContext(ctx) + g.Go(func() error { + // Re-write the intent with a higher timestamp. + txn.WriteTimestamp = ts3 + txn.Sequence = 2 + // Use a batch since MVCCPut is not atomic when using an Engine and we + // are not using latches to prevent a concurrent read in the other + // goroutine. A non-atomic Put can cause the strict invariant checking + // in intentInterleavingIter to be violated. + b := e.NewBatch() + defer b.Close() + if err := MVCCPut(ctx, b, nil, kA, ts1, hlc.ClockTimestamp{}, vA2, txn); err != nil { + return err + } + return b.Commit(false) + }) + g.Go(func() error { + // Iterate with a time range that includes the initial intent but does + // not include the new intent. + kvs, err := slurpKVsInTimeRange(e, kA, ts0, ts2) + + // There are two permissible outcomes from the scan. If the iteration + // wins the race with the put that moves the intent then it should + // observe the intent and return a write intent error. If the iteration + // loses the race with the put that moves the intent then it should + // observe and return nothing because there will be no committed or + // provisional keys in its time range. + if err != nil { + if !testutils.IsError(err, `conflicting intents on "kA"`) { + return err } - }) + } else { + if len(kvs) != 0 { + return errors.Errorf(`unexpected kvs: %v`, kvs) + } + } + return nil + }) + if err := g.Wait(); err != nil { + t.Fatal(err) } } diff --git a/pkg/storage/mvcc_logical_ops_test.go b/pkg/storage/mvcc_logical_ops_test.go index 2f57b224d9ba..b045c1b3e3ed 100644 --- a/pkg/storage/mvcc_logical_ops_test.go +++ b/pkg/storage/mvcc_logical_ops_test.go @@ -29,146 +29,142 @@ func TestMVCCOpLogWriter(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - batch := engine.NewBatch() - ol := NewOpLoggerBatch(batch) - defer ol.Close() + batch := engine.NewBatch() + ol := NewOpLoggerBatch(batch) + defer ol.Close() - // Write a value and an intent. - if err := MVCCPut(ctx, ol, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - txn1ts := makeTxn(*txn1, hlc.Timestamp{Logical: 2}) - if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { - t.Fatal(err) - } + // Write a value and an intent. + if err := MVCCPut(ctx, ol, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + txn1ts := makeTxn(*txn1, hlc.Timestamp{Logical: 2}) + if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + t.Fatal(err) + } - // Write a value and an intent on local keys. - localKey := keys.MakeRangeIDPrefix(1) - if err := MVCCPut(ctx, ol, nil, localKey, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { - t.Fatal(err) - } + // Write a value and an intent on local keys. + localKey := keys.MakeRangeIDPrefix(1) + if err := MVCCPut(ctx, ol, nil, localKey, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + t.Fatal(err) + } - // Update the intents and write another. - txn1ts.Sequence++ - txn1ts.WriteTimestamp = hlc.Timestamp{Logical: 3} - if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { - t.Fatal(err) - } - // Set the txn timestamp to a larger value than the intent. - txn1LargerTS := makeTxn(*txn1, hlc.Timestamp{Logical: 4}) - txn1LargerTS.WriteTimestamp = hlc.Timestamp{Logical: 4} - if err := MVCCPut(ctx, ol, nil, testKey2, txn1LargerTS.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1LargerTS); err != nil { - t.Fatal(err) - } + // Update the intents and write another. + txn1ts.Sequence++ + txn1ts.WriteTimestamp = hlc.Timestamp{Logical: 3} + if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + t.Fatal(err) + } + // Set the txn timestamp to a larger value than the intent. + txn1LargerTS := makeTxn(*txn1, hlc.Timestamp{Logical: 4}) + txn1LargerTS.WriteTimestamp = hlc.Timestamp{Logical: 4} + if err := MVCCPut(ctx, ol, nil, testKey2, txn1LargerTS.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1LargerTS); err != nil { + t.Fatal(err) + } - // Resolve all three intent. - txn1CommitTS := *txn1Commit - txn1CommitTS.WriteTimestamp = hlc.Timestamp{Logical: 4} - if _, _, err := MVCCResolveWriteIntentRange(ctx, ol, nil, - roachpb.MakeLockUpdate( - &txn1CommitTS, - roachpb.Span{Key: testKey1, EndKey: testKey2.Next()}), - math.MaxInt64); err != nil { - t.Fatal(err) - } - if _, _, err := MVCCResolveWriteIntentRange(ctx, ol, nil, - roachpb.MakeLockUpdate( - &txn1CommitTS, - roachpb.Span{Key: localKey, EndKey: localKey.Next()}), - math.MaxInt64); err != nil { - t.Fatal(err) - } + // Resolve all three intent. + txn1CommitTS := *txn1Commit + txn1CommitTS.WriteTimestamp = hlc.Timestamp{Logical: 4} + if _, _, err := MVCCResolveWriteIntentRange(ctx, ol, nil, + roachpb.MakeLockUpdate( + &txn1CommitTS, + roachpb.Span{Key: testKey1, EndKey: testKey2.Next()}), + math.MaxInt64); err != nil { + t.Fatal(err) + } + if _, _, err := MVCCResolveWriteIntentRange(ctx, ol, nil, + roachpb.MakeLockUpdate( + &txn1CommitTS, + roachpb.Span{Key: localKey, EndKey: localKey.Next()}), + math.MaxInt64); err != nil { + t.Fatal(err) + } - // Write another intent, push it, then abort it. - txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 5}) - if err := MVCCPut(ctx, ol, nil, testKey3, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value4, txn2ts); err != nil { - t.Fatal(err) - } - txn2Pushed := *txn2 - txn2Pushed.WriteTimestamp = hlc.Timestamp{Logical: 6} - if _, err := MVCCResolveWriteIntent(ctx, ol, nil, - roachpb.MakeLockUpdate(&txn2Pushed, roachpb.Span{Key: testKey3}), - ); err != nil { - t.Fatal(err) - } - txn2Abort := txn2Pushed - txn2Abort.Status = roachpb.ABORTED - if _, err := MVCCResolveWriteIntent(ctx, ol, nil, - roachpb.MakeLockUpdate(&txn2Abort, roachpb.Span{Key: testKey3}), - ); err != nil { - t.Fatal(err) - } + // Write another intent, push it, then abort it. + txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 5}) + if err := MVCCPut(ctx, ol, nil, testKey3, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value4, txn2ts); err != nil { + t.Fatal(err) + } + txn2Pushed := *txn2 + txn2Pushed.WriteTimestamp = hlc.Timestamp{Logical: 6} + if _, err := MVCCResolveWriteIntent(ctx, ol, nil, + roachpb.MakeLockUpdate(&txn2Pushed, roachpb.Span{Key: testKey3}), + ); err != nil { + t.Fatal(err) + } + txn2Abort := txn2Pushed + txn2Abort.Status = roachpb.ABORTED + if _, err := MVCCResolveWriteIntent(ctx, ol, nil, + roachpb.MakeLockUpdate(&txn2Abort, roachpb.Span{Key: testKey3}), + ); err != nil { + t.Fatal(err) + } - // Write an inline value. This should be ignored by the log. - if err := MVCCPut(ctx, ol, nil, testKey6, hlc.Timestamp{}, hlc.ClockTimestamp{}, value6, nil); err != nil { - t.Fatal(err) - } + // Write an inline value. This should be ignored by the log. + if err := MVCCPut(ctx, ol, nil, testKey6, hlc.Timestamp{}, hlc.ClockTimestamp{}, value6, nil); err != nil { + t.Fatal(err) + } - // Verify that the recorded logical ops match expectations. - makeOp := func(val interface{}) enginepb.MVCCLogicalOp { - var op enginepb.MVCCLogicalOp - op.MustSetValue(val) - return op - } - exp := []enginepb.MVCCLogicalOp{ - makeOp(&enginepb.MVCCWriteValueOp{ - Key: testKey1.Clone(), - Timestamp: hlc.Timestamp{Logical: 1}, - }), - makeOp(&enginepb.MVCCWriteIntentOp{ - TxnID: txn1.ID, - TxnKey: txn1.Key, - TxnMinTimestamp: txn1.MinTimestamp, - Timestamp: hlc.Timestamp{Logical: 2}, - }), - makeOp(&enginepb.MVCCUpdateIntentOp{ - TxnID: txn1.ID, - Timestamp: hlc.Timestamp{Logical: 3}, - }), - makeOp(&enginepb.MVCCWriteIntentOp{ - TxnID: txn1.ID, - TxnKey: txn1.Key, - TxnMinTimestamp: txn1.MinTimestamp, - Timestamp: hlc.Timestamp{Logical: 4}, - }), - makeOp(&enginepb.MVCCCommitIntentOp{ - TxnID: txn1.ID, - Key: testKey1.Clone(), - Timestamp: hlc.Timestamp{Logical: 4}, - }), - makeOp(&enginepb.MVCCCommitIntentOp{ - TxnID: txn1.ID, - Key: testKey2.Clone(), - Timestamp: hlc.Timestamp{Logical: 4}, - }), - makeOp(&enginepb.MVCCWriteIntentOp{ - TxnID: txn2.ID, - TxnKey: txn2.Key, - TxnMinTimestamp: txn2.MinTimestamp, - Timestamp: hlc.Timestamp{Logical: 5}, - }), - makeOp(&enginepb.MVCCUpdateIntentOp{ - TxnID: txn2.ID, - Timestamp: hlc.Timestamp{Logical: 6}, - }), - makeOp(&enginepb.MVCCAbortIntentOp{ - TxnID: txn2.ID, - }), - } - if diff := pretty.Diff(exp, ol.LogicalOps()); diff != nil { - t.Errorf("unexpected logical op differences:\n%s", strings.Join(diff, "\n")) - } - }) + // Verify that the recorded logical ops match expectations. + makeOp := func(val interface{}) enginepb.MVCCLogicalOp { + var op enginepb.MVCCLogicalOp + op.MustSetValue(val) + return op + } + exp := []enginepb.MVCCLogicalOp{ + makeOp(&enginepb.MVCCWriteValueOp{ + Key: testKey1.Clone(), + Timestamp: hlc.Timestamp{Logical: 1}, + }), + makeOp(&enginepb.MVCCWriteIntentOp{ + TxnID: txn1.ID, + TxnKey: txn1.Key, + TxnMinTimestamp: txn1.MinTimestamp, + Timestamp: hlc.Timestamp{Logical: 2}, + }), + makeOp(&enginepb.MVCCUpdateIntentOp{ + TxnID: txn1.ID, + Timestamp: hlc.Timestamp{Logical: 3}, + }), + makeOp(&enginepb.MVCCWriteIntentOp{ + TxnID: txn1.ID, + TxnKey: txn1.Key, + TxnMinTimestamp: txn1.MinTimestamp, + Timestamp: hlc.Timestamp{Logical: 4}, + }), + makeOp(&enginepb.MVCCCommitIntentOp{ + TxnID: txn1.ID, + Key: testKey1.Clone(), + Timestamp: hlc.Timestamp{Logical: 4}, + }), + makeOp(&enginepb.MVCCCommitIntentOp{ + TxnID: txn1.ID, + Key: testKey2.Clone(), + Timestamp: hlc.Timestamp{Logical: 4}, + }), + makeOp(&enginepb.MVCCWriteIntentOp{ + TxnID: txn2.ID, + TxnKey: txn2.Key, + TxnMinTimestamp: txn2.MinTimestamp, + Timestamp: hlc.Timestamp{Logical: 5}, + }), + makeOp(&enginepb.MVCCUpdateIntentOp{ + TxnID: txn2.ID, + Timestamp: hlc.Timestamp{Logical: 6}, + }), + makeOp(&enginepb.MVCCAbortIntentOp{ + TxnID: txn2.ID, + }), + } + if diff := pretty.Diff(exp, ol.LogicalOps()); diff != nil { + t.Errorf("unexpected logical op differences:\n%s", strings.Join(diff, "\n")) } } diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index fafb064e831c..c46f7ec8f345 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -85,87 +85,83 @@ var emptyMVCCValueHeaderSize = func() int64 { func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} - - assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - - key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} - // Put a value. - value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, nil); err != nil { - t.Fatal(err) - } + engine := NewDefaultInMemForTesting() + defer engine.Close() - mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - vKeySize := MVCCVersionTimestampSize // 12 - vValSize := int64(len(value.RawBytes)) // 10 - if disableSimpleValueEncoding { - vValSize += emptyMVCCValueHeaderSize // 17 - } + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - expMS := enginepb.MVCCStats{ - LiveBytes: mKeySize + vKeySize + vValSize, // 24[+7] - LiveCount: 1, - KeyBytes: mKeySize + vKeySize, // 14 - KeyCount: 1, - ValBytes: vValSize, // 10[+7] - ValCount: 1, - LastUpdateNanos: 1e9, - } - assertEq(t, engine, "after put", aggMS, &expMS) + assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - // Delete the value at ts=3. We'll commit this at ts=4 later. - ts3 := hlc.Timestamp{WallTime: 3 * 1e9} - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts3}, - ReadTimestamp: ts3, - } - if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { - t.Fatal(err) - } + key := roachpb.Key("a") + ts1 := hlc.Timestamp{WallTime: 1e9} + // Put a value. + value := roachpb.MakeValueFromString("value") + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, nil); err != nil { + t.Fatal(err) + } - // Now commit the value, but with a timestamp gap (i.e. this is a - // push-commit as it would happen for a SNAPSHOT txn) - ts4 := hlc.Timestamp{WallTime: 4 * 1e9} - txn.Status = roachpb.COMMITTED - txn.WriteTimestamp.Forward(ts4) - if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, - roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}), - ); err != nil { - t.Fatal(err) - } + mKeySize := int64(mvccKey(key).EncodedSize()) // 2 + vKeySize := MVCCVersionTimestampSize // 12 + vValSize := int64(len(value.RawBytes)) // 10 + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize // 17 + } - // The initial write used the simple MVCCValue encoding. When resolved to - // a higher timestamp, the MVCCValue retained its local timestamp, meaning - // that it now uses the extended MVCCValue encoding. - vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} - vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 - vValSize += vValHeaderSize // 23 - - expAggMS := enginepb.MVCCStats{ - LastUpdateNanos: 4e9, - LiveBytes: 0, - LiveCount: 0, - KeyCount: 1, - ValCount: 2, - // The implicit meta record (deletion tombstone) counts for len("a")+1=2. - // Two versioned keys count for 2*vKeySize. - KeyBytes: mKeySize + 2*vKeySize, - ValBytes: vValSize, - // No GCBytesAge has been accrued yet, as the value just got non-live at 4s. - GCBytesAge: 0, - } + expMS := enginepb.MVCCStats{ + LiveBytes: mKeySize + vKeySize + vValSize, // 24[+7] + LiveCount: 1, + KeyBytes: mKeySize + vKeySize, // 14 + KeyCount: 1, + ValBytes: vValSize, // 10[+7] + ValCount: 1, + LastUpdateNanos: 1e9, + } + assertEq(t, engine, "after put", aggMS, &expMS) - assertEq(t, engine, "after committing", aggMS, &expAggMS) - }) + // Delete the value at ts=3. We'll commit this at ts=4 later. + ts3 := hlc.Timestamp{WallTime: 3 * 1e9} + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts3}, + ReadTimestamp: ts3, + } + if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + t.Fatal(err) + } + + // Now commit the value, but with a timestamp gap (i.e. this is a + // push-commit as it would happen for a SNAPSHOT txn) + ts4 := hlc.Timestamp{WallTime: 4 * 1e9} + txn.Status = roachpb.COMMITTED + txn.WriteTimestamp.Forward(ts4) + if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, + roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}), + ); err != nil { + t.Fatal(err) } + + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize += vValHeaderSize // 23 + + expAggMS := enginepb.MVCCStats{ + LastUpdateNanos: 4e9, + LiveBytes: 0, + LiveCount: 0, + KeyCount: 1, + ValCount: 2, + // The implicit meta record (deletion tombstone) counts for len("a")+1=2. + // Two versioned keys count for 2*vKeySize. + KeyBytes: mKeySize + 2*vKeySize, + ValBytes: vValSize, + // No GCBytesAge has been accrued yet, as the value just got non-live at 4s. + GCBytesAge: 0, + } + + assertEq(t, engine, "after committing", aggMS, &expAggMS) } // TestMVCCStatsPutCommitMovesTimestamp is similar to @@ -174,90 +170,86 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + engine := NewDefaultInMemForTesting() + defer engine.Close() - assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, - ReadTimestamp: ts1, - } - // Write an intent at t=1s. - value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, txn); err != nil { - t.Fatal(err) - } + assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - mValSize := int64((&enginepb.MVCCMetadata{ // 46 - Timestamp: ts1.ToLegacyTimestamp(), - Deleted: false, - Txn: &txn.TxnMeta, - }).Size()) - mValSize += 2 - vKeySize := MVCCVersionTimestampSize // 12 - vValSize := int64(len(value.RawBytes)) // 10 - if disableSimpleValueEncoding { - vValSize += emptyMVCCValueHeaderSize // 17 - } + key := roachpb.Key("a") + ts1 := hlc.Timestamp{WallTime: 1e9} + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, + ReadTimestamp: ts1, + } + // Write an intent at t=1s. + value := roachpb.MakeValueFromString("value") + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, txn); err != nil { + t.Fatal(err) + } - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 68[+2][+7] - LiveCount: 1, - KeyBytes: mKeySize + vKeySize, // 2+12 =14 - KeyCount: 1, - ValBytes: mValSize + vValSize, // (46[+2])+(10[+7]) = 54[+2][+7] - ValCount: 1, - IntentCount: 1, - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] - GCBytesAge: 0, - } - assertEq(t, engine, "after put", aggMS, &expMS) - - // Now commit the intent, but with a timestamp gap (i.e. this is a - // push-commit as it would happen for a SNAPSHOT txn) - ts4 := hlc.Timestamp{WallTime: 4 * 1e9} - txn.Status = roachpb.COMMITTED - txn.WriteTimestamp.Forward(ts4) - if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, - roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}), - ); err != nil { - t.Fatal(err) - } + mKeySize := int64(mvccKey(key).EncodedSize()) // 2 + mValSize := int64((&enginepb.MVCCMetadata{ // 46 + Timestamp: ts1.ToLegacyTimestamp(), + Deleted: false, + Txn: &txn.TxnMeta, + }).Size()) + mValSize += 2 + vKeySize := MVCCVersionTimestampSize // 12 + vValSize := int64(len(value.RawBytes)) // 10 + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize // 17 + } - // The initial write used the simple MVCCValue encoding. When resolved to - // a higher timestamp, the MVCCValue retained its local timestamp, meaning - // that it now uses the extended MVCCValue encoding. - vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} - vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 - vValSize = int64(len(value.RawBytes)) + vValHeaderSize // 23 - - expAggMS := enginepb.MVCCStats{ - LastUpdateNanos: 4e9, - LiveBytes: mKeySize + vKeySize + vValSize, // 2+12+23 = 37 - LiveCount: 1, - KeyCount: 1, - ValCount: 1, - // The implicit meta record counts for len("a")+1=2. - // One versioned key counts for vKeySize. - KeyBytes: mKeySize + vKeySize, - ValBytes: vValSize, - GCBytesAge: 0, // this was once erroneously negative - } + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 68[+2][+7] + LiveCount: 1, + KeyBytes: mKeySize + vKeySize, // 2+12 =14 + KeyCount: 1, + ValBytes: mValSize + vValSize, // (46[+2])+(10[+7]) = 54[+2][+7] + ValCount: 1, + IntentCount: 1, + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] + GCBytesAge: 0, + } + assertEq(t, engine, "after put", aggMS, &expMS) + + // Now commit the intent, but with a timestamp gap (i.e. this is a + // push-commit as it would happen for a SNAPSHOT txn) + ts4 := hlc.Timestamp{WallTime: 4 * 1e9} + txn.Status = roachpb.COMMITTED + txn.WriteTimestamp.Forward(ts4) + if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, + roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}), + ); err != nil { + t.Fatal(err) + } - assertEq(t, engine, "after committing", aggMS, &expAggMS) - }) + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize = int64(len(value.RawBytes)) + vValHeaderSize // 23 + + expAggMS := enginepb.MVCCStats{ + LastUpdateNanos: 4e9, + LiveBytes: mKeySize + vKeySize + vValSize, // 2+12+23 = 37 + LiveCount: 1, + KeyCount: 1, + ValCount: 1, + // The implicit meta record counts for len("a")+1=2. + // One versioned key counts for vKeySize. + KeyBytes: mKeySize + vKeySize, + ValBytes: vValSize, + GCBytesAge: 0, // this was once erroneously negative } + + assertEq(t, engine, "after committing", aggMS, &expAggMS) } // TestMVCCStatsPutPushMovesTimestamp is similar to TestMVCCStatsPutCommitMovesTimestamp: @@ -266,95 +258,91 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + engine := NewDefaultInMemForTesting() + defer engine.Close() - assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, - ReadTimestamp: ts1, - } - // Write an intent. - value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { - t.Fatal(err) - } + assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - mValSize := int64((&enginepb.MVCCMetadata{ // 46 - Timestamp: ts1.ToLegacyTimestamp(), - Deleted: false, - Txn: &txn.TxnMeta, - }).Size()) - mValSize += 2 - vKeySize := MVCCVersionTimestampSize // 12 - vValSize := int64(len(value.RawBytes)) // 10 - if disableSimpleValueEncoding { - vValSize += emptyMVCCValueHeaderSize // 17 - } + key := roachpb.Key("a") + ts1 := hlc.Timestamp{WallTime: 1e9} + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, + ReadTimestamp: ts1, + } + // Write an intent. + value := roachpb.MakeValueFromString("value") + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { + t.Fatal(err) + } - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 70[+2][+7] - LiveCount: 1, - KeyBytes: mKeySize + vKeySize, // 2+12 = 14 - KeyCount: 1, - ValBytes: mValSize + vValSize, // (46[+2])+(10[+7]) = 54[+2][+7] - ValCount: 1, - IntentAge: 0, - IntentCount: 1, - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] - } - assertEq(t, engine, "after put", aggMS, &expMS) - - // Now push the value, but with a timestamp gap (i.e. this is a - // push as it would happen for a SNAPSHOT txn) - ts4 := hlc.Timestamp{WallTime: 4 * 1e9} - txn.WriteTimestamp.Forward(ts4) - if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, - roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}), - ); err != nil { - t.Fatal(err) - } - // Account for removal of TxnDidNotUpdateMeta. - mValSize -= 2 - - // The initial write used the simple MVCCValue encoding. When resolved to - // a higher timestamp, the MVCCValue retained its local timestamp, meaning - // that it now uses the extended MVCCValue encoding. - vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} - vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 - vValSize = int64(len(value.RawBytes)) + vValHeaderSize // 23 - - expAggMS := enginepb.MVCCStats{ - LastUpdateNanos: 4e9, - LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+54+12+23 = 91 - LiveCount: 1, - KeyCount: 1, - ValCount: 1, - // The explicit meta record counts for len("a")+1=2. - // One versioned key counts for vKeySize. - KeyBytes: mKeySize + vKeySize, - // The intent is still there, so we see mValSize. - ValBytes: mValSize + vValSize, // 54+23 = 69 - IntentAge: 0, // this was once erroneously positive - IntentCount: 1, // still there - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // still there - } + mKeySize := int64(mvccKey(key).EncodedSize()) // 2 + mValSize := int64((&enginepb.MVCCMetadata{ // 46 + Timestamp: ts1.ToLegacyTimestamp(), + Deleted: false, + Txn: &txn.TxnMeta, + }).Size()) + mValSize += 2 + vKeySize := MVCCVersionTimestampSize // 12 + vValSize := int64(len(value.RawBytes)) // 10 + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize // 17 + } - assertEq(t, engine, "after pushing", aggMS, &expAggMS) - }) + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 70[+2][+7] + LiveCount: 1, + KeyBytes: mKeySize + vKeySize, // 2+12 = 14 + KeyCount: 1, + ValBytes: mValSize + vValSize, // (46[+2])+(10[+7]) = 54[+2][+7] + ValCount: 1, + IntentAge: 0, + IntentCount: 1, + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] + } + assertEq(t, engine, "after put", aggMS, &expMS) + + // Now push the value, but with a timestamp gap (i.e. this is a + // push as it would happen for a SNAPSHOT txn) + ts4 := hlc.Timestamp{WallTime: 4 * 1e9} + txn.WriteTimestamp.Forward(ts4) + if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, + roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}), + ); err != nil { + t.Fatal(err) + } + // Account for removal of TxnDidNotUpdateMeta. + mValSize -= 2 + + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vValHeader := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts1)} + vValHeaderSize := extendedPreludeSize + int64(vValHeader.Size()) // 13 + vValSize = int64(len(value.RawBytes)) + vValHeaderSize // 23 + + expAggMS := enginepb.MVCCStats{ + LastUpdateNanos: 4e9, + LiveBytes: mKeySize + mValSize + vKeySize + vValSize, // 2+54+12+23 = 91 + LiveCount: 1, + KeyCount: 1, + ValCount: 1, + // The explicit meta record counts for len("a")+1=2. + // One versioned key counts for vKeySize. + KeyBytes: mKeySize + vKeySize, + // The intent is still there, so we see mValSize. + ValBytes: mValSize + vValSize, // 54+23 = 69 + IntentAge: 0, // this was once erroneously positive + IntentCount: 1, // still there + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vValSize, // still there } + + assertEq(t, engine, "after pushing", aggMS, &expAggMS) } // TestMVCCStatsDeleteMovesTimestamp is similar to TestMVCCStatsPutCommitMovesTimestamp: @@ -363,132 +351,128 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + engine := NewDefaultInMemForTesting() + defer engine.Close() - assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2 * 1e9} + assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - key := roachpb.Key("a") - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, - ReadTimestamp: ts1, - } + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2 * 1e9} - // Write an intent. - value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { - t.Fatal(err) - } + key := roachpb.Key("a") + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, + ReadTimestamp: ts1, + } - mKeySize := int64(mvccKey(key).EncodedSize()) - require.EqualValues(t, mKeySize, 2) - - mVal1Size := int64((&enginepb.MVCCMetadata{ - Timestamp: ts1.ToLegacyTimestamp(), - Deleted: false, - Txn: &txn.TxnMeta, - }).Size()) - require.EqualValues(t, mVal1Size, 46) - mVal1Size += 2 - - // TODO(sumeer): this is the first put at ts1, so why are we using this m1ValSize - // instead of mVal1Size being sufficient? - m1ValSize := int64((&enginepb.MVCCMetadata{ - Timestamp: ts2.ToLegacyTimestamp(), - Deleted: false, - Txn: &txn.TxnMeta, - }).Size()) - require.EqualValues(t, m1ValSize, 46) - m1ValSize += 2 - - vKeySize := MVCCVersionTimestampSize - require.EqualValues(t, vKeySize, 12) - - vValSize := int64(len(value.RawBytes)) - require.EqualValues(t, vValSize, 10) - if disableSimpleValueEncoding { - vValSize += emptyMVCCValueHeaderSize - require.EqualValues(t, vValSize, 17) - } + // Write an intent. + value := roachpb.MakeValueFromString("value") + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { + t.Fatal(err) + } - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 70[+2][+7] - LiveCount: 1, - KeyBytes: mKeySize + vKeySize, // 2+12 = 14 - KeyCount: 1, - ValBytes: mVal1Size + vValSize, // (46[+2])+(10[+7]) = 56[+2][+7] - ValCount: 1, - IntentAge: 0, - IntentCount: 1, - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] - } - assertEq(t, engine, "after put", aggMS, &expMS) - - // Now replace our intent with a deletion intent, but with a timestamp gap. - // This could happen if a transaction got restarted with a higher timestamp - // and ran logic different from that in the first attempt. - txn.WriteTimestamp.Forward(ts2) - - txn.Sequence++ - - // Annoyingly, the new meta value is actually a little larger thanks to the - // sequence number. Also since there was a write previously on the same - // transaction, the IntentHistory will add a few bytes to the metadata. - encValue, err := EncodeMVCCValue(MVCCValue{Value: value}) - require.NoError(t, err) - m2ValSize := int64((&enginepb.MVCCMetadata{ - Timestamp: ts2.ToLegacyTimestamp(), - Txn: &txn.TxnMeta, - IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: encValue}, - }, - }).Size()) - expM2ValSize := 64 - if disableSimpleValueEncoding { - expM2ValSize = 71 - } - require.EqualValues(t, m2ValSize, expM2ValSize) + mKeySize := int64(mvccKey(key).EncodedSize()) + require.EqualValues(t, mKeySize, 2) + + mVal1Size := int64((&enginepb.MVCCMetadata{ + Timestamp: ts1.ToLegacyTimestamp(), + Deleted: false, + Txn: &txn.TxnMeta, + }).Size()) + require.EqualValues(t, mVal1Size, 46) + mVal1Size += 2 + + // TODO(sumeer): this is the first put at ts1, so why are we using this m1ValSize + // instead of mVal1Size being sufficient? + m1ValSize := int64((&enginepb.MVCCMetadata{ + Timestamp: ts2.ToLegacyTimestamp(), + Deleted: false, + Txn: &txn.TxnMeta, + }).Size()) + require.EqualValues(t, m1ValSize, 46) + m1ValSize += 2 + + vKeySize := MVCCVersionTimestampSize + require.EqualValues(t, vKeySize, 12) + + vValSize := int64(len(value.RawBytes)) + require.EqualValues(t, vValSize, 10) + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize + require.EqualValues(t, vValSize, 17) + } - if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { - t.Fatal(err) - } + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 70[+2][+7] + LiveCount: 1, + KeyBytes: mKeySize + vKeySize, // 2+12 = 14 + KeyCount: 1, + ValBytes: mVal1Size + vValSize, // (46[+2])+(10[+7]) = 56[+2][+7] + ValCount: 1, + IntentAge: 0, + IntentCount: 1, + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] + } + assertEq(t, engine, "after put", aggMS, &expMS) + + // Now replace our intent with a deletion intent, but with a timestamp gap. + // This could happen if a transaction got restarted with a higher timestamp + // and ran logic different from that in the first attempt. + txn.WriteTimestamp.Forward(ts2) + + txn.Sequence++ + + // Annoyingly, the new meta value is actually a little larger thanks to the + // sequence number. Also since there was a write previously on the same + // transaction, the IntentHistory will add a few bytes to the metadata. + encValue, err := EncodeMVCCValue(MVCCValue{Value: value}) + require.NoError(t, err) + m2ValSize := int64((&enginepb.MVCCMetadata{ + Timestamp: ts2.ToLegacyTimestamp(), + Txn: &txn.TxnMeta, + IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ + {Sequence: 0, Value: encValue}, + }, + }).Size()) + expM2ValSize := 64 + if disableSimpleValueEncoding { + expM2ValSize = 71 + } + require.EqualValues(t, m2ValSize, expM2ValSize) - vVal2Size := int64(0) // tombstone - if disableSimpleValueEncoding { - vVal2Size = emptyMVCCValueHeaderSize // 7 - } + if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + t.Fatal(err) + } - expAggMS := enginepb.MVCCStats{ - LastUpdateNanos: 2e9, - LiveBytes: 0, - LiveCount: 0, - KeyCount: 1, - ValCount: 1, - // The explicit meta record counts for len("a")+1=2. - // One versioned key counts for vKeySize. - KeyBytes: mKeySize + vKeySize, - // The intent is still there, but this time with mVal2Size, and a zero vValSize. - ValBytes: m2ValSize + vVal2Size, - IntentAge: 0, - IntentCount: 1, // still there - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vVal2Size, // still there, but now without vValSize - GCBytesAge: 0, // this was once erroneously negative - } + vVal2Size := int64(0) // tombstone + if disableSimpleValueEncoding { + vVal2Size = emptyMVCCValueHeaderSize // 7 + } - assertEq(t, engine, "after deleting", aggMS, &expAggMS) - }) + expAggMS := enginepb.MVCCStats{ + LastUpdateNanos: 2e9, + LiveBytes: 0, + LiveCount: 0, + KeyCount: 1, + ValCount: 1, + // The explicit meta record counts for len("a")+1=2. + // One versioned key counts for vKeySize. + KeyBytes: mKeySize + vKeySize, + // The intent is still there, but this time with mVal2Size, and a zero vValSize. + ValBytes: m2ValSize + vVal2Size, + IntentAge: 0, + IntentCount: 1, // still there + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vVal2Size, // still there, but now without vValSize + GCBytesAge: 0, // this was once erroneously negative } + + assertEq(t, engine, "after deleting", aggMS, &expAggMS) } // TestMVCCStatsPutMovesDeletionTimestamp is similar to TestMVCCStatsPutCommitMovesTimestamp: A @@ -497,124 +481,121 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + engine := NewDefaultInMemForTesting() + defer engine.Close() - assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2 * 1e9} + assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - key := roachpb.Key("a") - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, - ReadTimestamp: ts1, - } + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2 * 1e9} - // Write a deletion tombstone intent. - if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { - t.Fatal(err) - } + key := roachpb.Key("a") + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, + ReadTimestamp: ts1, + } - mKeySize := int64(mvccKey(key).EncodedSize()) - require.EqualValues(t, mKeySize, 2) + // Write a deletion tombstone intent. + if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + t.Fatal(err) + } - mVal1Size := int64((&enginepb.MVCCMetadata{ - Timestamp: ts1.ToLegacyTimestamp(), - Deleted: false, - Txn: &txn.TxnMeta, - }).Size()) - require.EqualValues(t, mVal1Size, 46) - mVal1Size += 2 + mKeySize := int64(mvccKey(key).EncodedSize()) + require.EqualValues(t, mKeySize, 2) - vKeySize := MVCCVersionTimestampSize - require.EqualValues(t, vKeySize, 12) + mVal1Size := int64((&enginepb.MVCCMetadata{ + Timestamp: ts1.ToLegacyTimestamp(), + Deleted: false, + Txn: &txn.TxnMeta, + }).Size()) + require.EqualValues(t, mVal1Size, 46) + mVal1Size += 2 - vVal1Size := int64(0) // tombstone - if disableSimpleValueEncoding { - vVal1Size = emptyMVCCValueHeaderSize // 7 - } + vKeySize := MVCCVersionTimestampSize + require.EqualValues(t, vKeySize, 12) - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - LiveBytes: 0, - LiveCount: 0, - KeyBytes: mKeySize + vKeySize, // 2 + 12 = 24 - KeyCount: 1, - ValBytes: mVal1Size + vVal1Size, // 46[+2] [+7] - ValCount: 1, - IntentAge: 0, - IntentCount: 1, - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vVal1Size, // 12 [+7] - GCBytesAge: 0, - } - assertEq(t, engine, "after delete", aggMS, &expMS) - - // Now replace our deletion with a value intent, but with a timestamp gap. - // This could happen if a transaction got restarted with a higher timestamp - // and ran logic different from that in the first attempt. - txn.WriteTimestamp.Forward(ts2) - - txn.Sequence++ - - // Annoyingly, the new meta value is actually a little larger thanks to the - // sequence number. Also the value is larger because the previous intent on the - // transaction is recorded in the IntentHistory. - encVal1, err := EncodeMVCCValue(MVCCValue{Value: roachpb.Value{RawBytes: []byte{}}}) - require.NoError(t, err) - mVal2Size := int64((&enginepb.MVCCMetadata{ - Timestamp: ts2.ToLegacyTimestamp(), - Txn: &txn.TxnMeta, - IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: encVal1}, - }, - }).Size()) - expMVal2Size := 54 - if disableSimpleValueEncoding { - expMVal2Size = 61 - } - require.EqualValues(t, mVal2Size, expMVal2Size) + vVal1Size := int64(0) // tombstone + if disableSimpleValueEncoding { + vVal1Size = emptyMVCCValueHeaderSize // 7 + } - value := roachpb.MakeValueFromString("value") + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + LiveBytes: 0, + LiveCount: 0, + KeyBytes: mKeySize + vKeySize, // 2 + 12 = 24 + KeyCount: 1, + ValBytes: mVal1Size + vVal1Size, // 46[+2] [+7] + ValCount: 1, + IntentAge: 0, + IntentCount: 1, + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vVal1Size, // 12 [+7] + GCBytesAge: 0, + } + assertEq(t, engine, "after delete", aggMS, &expMS) + + // Now replace our deletion with a value intent, but with a timestamp gap. + // This could happen if a transaction got restarted with a higher timestamp + // and ran logic different from that in the first attempt. + txn.WriteTimestamp.Forward(ts2) + + txn.Sequence++ + + // Annoyingly, the new meta value is actually a little larger thanks to the + // sequence number. Also the value is larger because the previous intent on the + // transaction is recorded in the IntentHistory. + encVal1, err := EncodeMVCCValue(MVCCValue{Value: roachpb.Value{RawBytes: []byte{}}}) + require.NoError(t, err) + mVal2Size := int64((&enginepb.MVCCMetadata{ + Timestamp: ts2.ToLegacyTimestamp(), + Txn: &txn.TxnMeta, + IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ + {Sequence: 0, Value: encVal1}, + }, + }).Size()) + expMVal2Size := 54 + if disableSimpleValueEncoding { + expMVal2Size = 61 + } + require.EqualValues(t, mVal2Size, expMVal2Size) - vVal2Size := int64(len(value.RawBytes)) - require.EqualValues(t, vVal2Size, 10) - if disableSimpleValueEncoding { - vVal2Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal2Size, 17) - } + value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { - t.Fatal(err) - } + vVal2Size := int64(len(value.RawBytes)) + require.EqualValues(t, vVal2Size, 10) + if disableSimpleValueEncoding { + vVal2Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal2Size, 17) + } - expAggMS := enginepb.MVCCStats{ - LastUpdateNanos: 2e9, - LiveBytes: mKeySize + mVal2Size + vKeySize + vVal2Size, // 2+46+12+(10[+7]) = 70[+7] - LiveCount: 1, - KeyCount: 1, - ValCount: 1, - // The explicit meta record counts for len("a")+1=2. - // One versioned key counts for vKeySize. - KeyBytes: mKeySize + vKeySize, - // The intent is still there, but this time with mVal2Size, and a zero vValSize. - ValBytes: vVal2Size + mVal2Size, // (10[+7])+46 = 56[+7] - IntentAge: 0, - IntentCount: 1, // still there - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vVal2Size, // still there, now bigger - GCBytesAge: 0, // this was once erroneously negative - } + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { + t.Fatal(err) + } - assertEq(t, engine, "after put", aggMS, &expAggMS) - }) + expAggMS := enginepb.MVCCStats{ + LastUpdateNanos: 2e9, + LiveBytes: mKeySize + mVal2Size + vKeySize + vVal2Size, // 2+46+12+(10[+7]) = 70[+7] + LiveCount: 1, + KeyCount: 1, + ValCount: 1, + // The explicit meta record counts for len("a")+1=2. + // One versioned key counts for vKeySize. + KeyBytes: mKeySize + vKeySize, + // The intent is still there, but this time with mVal2Size, and a zero vValSize. + ValBytes: vVal2Size + mVal2Size, // (10[+7])+46 = 56[+7] + IntentAge: 0, + IntentCount: 1, // still there + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vVal2Size, // still there, now bigger + GCBytesAge: 0, // this was once erroneously negative } + + assertEq(t, engine, "after put", aggMS, &expAggMS) } // TestMVCCStatsDelDelCommit writes a non-transactional tombstone, and then adds an intent tombstone @@ -625,149 +606,145 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - key := roachpb.Key("a") + key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} - ts3 := hlc.Timestamp{WallTime: 3e9} + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2e9} + ts3 := hlc.Timestamp{WallTime: 3e9} - // Write a non-transactional tombstone at t=1s. - if _, err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } + // Write a non-transactional tombstone at t=1s. + if _, err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } - mKeySize := int64(mvccKey(key).EncodedSize()) - require.EqualValues(t, mKeySize, 2) - vKeySize := MVCCVersionTimestampSize - require.EqualValues(t, vKeySize, 12) + mKeySize := int64(mvccKey(key).EncodedSize()) + require.EqualValues(t, mKeySize, 2) + vKeySize := MVCCVersionTimestampSize + require.EqualValues(t, vKeySize, 12) - vValSize := int64(0) // tombstone - if disableSimpleValueEncoding { - vValSize = emptyMVCCValueHeaderSize // 7 - } + vValSize := int64(0) // tombstone + if disableSimpleValueEncoding { + vValSize = emptyMVCCValueHeaderSize // 7 + } - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - KeyBytes: mKeySize + vKeySize, - KeyCount: 1, - ValBytes: vValSize, - ValCount: 1, - } + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + KeyBytes: mKeySize + vKeySize, + KeyCount: 1, + ValBytes: vValSize, + ValCount: 1, + } - assertEq(t, engine, "after non-transactional delete", aggMS, &expMS) + assertEq(t, engine, "after non-transactional delete", aggMS, &expMS) - // Write a tombstone intent at t=2s. - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, - ReadTimestamp: ts2, - } - if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { - t.Fatal(err) - } + // Write a tombstone intent at t=2s. + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, + ReadTimestamp: ts2, + } + if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + t.Fatal(err) + } - mValSize := int64((&enginepb.MVCCMetadata{ - Timestamp: ts1.ToLegacyTimestamp(), - Deleted: true, - Txn: &txn.TxnMeta, - }).Size()) - require.EqualValues(t, mValSize, 46) - // Account for TxnDidNotUpdateMeta - mValSize += 2 - - expMS = enginepb.MVCCStats{ - LastUpdateNanos: 2e9, - KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 - KeyCount: 1, - ValBytes: mValSize + 2*vValSize, // 46[+2] [+7] - ValCount: 2, - IntentCount: 1, - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, - // The original non-transactional write (at 1s) has now aged one second. - GCBytesAge: 1 * (vKeySize + vValSize), - } - assertEq(t, engine, "after put", aggMS, &expMS) - - // Now commit or abort the intent, respectively, but with a timestamp gap - // (i.e. this is a push-commit as it would happen for a SNAPSHOT txn). - t.Run("Commit", func(t *testing.T) { - aggMS := *aggMS - engine := engine.NewBatch() - defer engine.Close() - - txnCommit := txn.Clone() - txnCommit.Status = roachpb.COMMITTED - txnCommit.WriteTimestamp.Forward(ts3) - if _, err := MVCCResolveWriteIntent(ctx, engine, &aggMS, - roachpb.MakeLockUpdate(txnCommit, roachpb.Span{Key: key}), - ); err != nil { - t.Fatal(err) - } + mValSize := int64((&enginepb.MVCCMetadata{ + Timestamp: ts1.ToLegacyTimestamp(), + Deleted: true, + Txn: &txn.TxnMeta, + }).Size()) + require.EqualValues(t, mValSize, 46) + // Account for TxnDidNotUpdateMeta + mValSize += 2 + + expMS = enginepb.MVCCStats{ + LastUpdateNanos: 2e9, + KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 + KeyCount: 1, + ValBytes: mValSize + 2*vValSize, // 46[+2] [+7] + ValCount: 2, + IntentCount: 1, + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vValSize, + // The original non-transactional write (at 1s) has now aged one second. + GCBytesAge: 1 * (vKeySize + vValSize), + } + assertEq(t, engine, "after put", aggMS, &expMS) + + // Now commit or abort the intent, respectively, but with a timestamp gap + // (i.e. this is a push-commit as it would happen for a SNAPSHOT txn). + t.Run("Commit", func(t *testing.T) { + aggMS := *aggMS + engine := engine.NewBatch() + defer engine.Close() + + txnCommit := txn.Clone() + txnCommit.Status = roachpb.COMMITTED + txnCommit.WriteTimestamp.Forward(ts3) + if _, err := MVCCResolveWriteIntent(ctx, engine, &aggMS, + roachpb.MakeLockUpdate(txnCommit, roachpb.Span{Key: key}), + ); err != nil { + t.Fatal(err) + } - // The initial write used the simple MVCCValue encoding. When resolved to - // a higher timestamp, the MVCCValue retained its local timestamp, meaning - // that it now uses the extended MVCCValue encoding. - vVal2Header := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts2)} - vVal2HeaderSize := extendedPreludeSize + int64(vVal2Header.Size()) // 13 - vVal2Size := vVal2HeaderSize + 0 // tombstone, so just a header - - expAggMS := enginepb.MVCCStats{ - LastUpdateNanos: 3e9, - KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 - KeyCount: 1, - ValBytes: vValSize + vVal2Size, - ValCount: 2, - IntentCount: 0, - IntentBytes: 0, - // The very first write picks up another second of age. Before a bug fix, - // this was failing to do so. - GCBytesAge: 2 * (vKeySize + vValSize), - } + // The initial write used the simple MVCCValue encoding. When resolved to + // a higher timestamp, the MVCCValue retained its local timestamp, meaning + // that it now uses the extended MVCCValue encoding. + vVal2Header := enginepb.MVCCValueHeader{LocalTimestamp: hlc.ClockTimestamp(ts2)} + vVal2HeaderSize := extendedPreludeSize + int64(vVal2Header.Size()) // 13 + vVal2Size := vVal2HeaderSize + 0 // tombstone, so just a header + + expAggMS := enginepb.MVCCStats{ + LastUpdateNanos: 3e9, + KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 + KeyCount: 1, + ValBytes: vValSize + vVal2Size, + ValCount: 2, + IntentCount: 0, + IntentBytes: 0, + // The very first write picks up another second of age. Before a bug fix, + // this was failing to do so. + GCBytesAge: 2 * (vKeySize + vValSize), + } - assertEq(t, engine, "after committing", &aggMS, &expAggMS) - }) - t.Run("Abort", func(t *testing.T) { - aggMS := *aggMS - engine := engine.NewBatch() - defer engine.Close() - - txnAbort := txn.Clone() - txnAbort.Status = roachpb.ABORTED - txnAbort.WriteTimestamp.Forward(ts3) - if _, err := MVCCResolveWriteIntent(ctx, engine, &aggMS, - roachpb.MakeLockUpdate(txnAbort, roachpb.Span{Key: key}), - ); err != nil { - t.Fatal(err) - } + assertEq(t, engine, "after committing", &aggMS, &expAggMS) + }) + t.Run("Abort", func(t *testing.T) { + aggMS := *aggMS + engine := engine.NewBatch() + defer engine.Close() + + txnAbort := txn.Clone() + txnAbort.Status = roachpb.ABORTED + txnAbort.WriteTimestamp.Forward(ts3) + if _, err := MVCCResolveWriteIntent(ctx, engine, &aggMS, + roachpb.MakeLockUpdate(txnAbort, roachpb.Span{Key: key}), + ); err != nil { + t.Fatal(err) + } - expAggMS := enginepb.MVCCStats{ - LastUpdateNanos: 3e9, - KeyBytes: mKeySize + vKeySize, // 2+12 = 14 - KeyCount: 1, - ValBytes: vValSize, - ValCount: 1, - IntentCount: 0, - IntentBytes: 0, - // We aborted our intent, but the value we first wrote was a tombstone, and - // so it's expected to retain its age. Since it's now the only value, it - // also contributes as a meta key. - GCBytesAge: 2 * (mKeySize + vKeySize + vValSize), - } + expAggMS := enginepb.MVCCStats{ + LastUpdateNanos: 3e9, + KeyBytes: mKeySize + vKeySize, // 2+12 = 14 + KeyCount: 1, + ValBytes: vValSize, + ValCount: 1, + IntentCount: 0, + IntentBytes: 0, + // We aborted our intent, but the value we first wrote was a tombstone, and + // so it's expected to retain its age. Since it's now the only value, it + // also contributes as a meta key. + GCBytesAge: 2 * (mKeySize + vKeySize + vValSize), + } - assertEq(t, engine, "after aborting", &aggMS, &expAggMS) - }) - }) - } + assertEq(t, engine, "after aborting", &aggMS, &expAggMS) + }) } // TestMVCCStatsPutDelPut is similar to TestMVCCStatsDelDelCommit, but its first @@ -780,189 +757,185 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - key := roachpb.Key("a") + key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} - ts3 := hlc.Timestamp{WallTime: 3e9} + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2e9} + ts3 := hlc.Timestamp{WallTime: 3e9} - // Write a non-transactional value at t=1s. - value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, nil); err != nil { - t.Fatal(err) - } + // Write a non-transactional value at t=1s. + value := roachpb.MakeValueFromString("value") + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, value, nil); err != nil { + t.Fatal(err) + } - mKeySize := int64(mvccKey(key).EncodedSize()) - require.EqualValues(t, mKeySize, 2) + mKeySize := int64(mvccKey(key).EncodedSize()) + require.EqualValues(t, mKeySize, 2) - vKeySize := MVCCVersionTimestampSize - require.EqualValues(t, vKeySize, 12) + vKeySize := MVCCVersionTimestampSize + require.EqualValues(t, vKeySize, 12) - vValSize := int64(len(value.RawBytes)) - require.EqualValues(t, vValSize, 10) - if disableSimpleValueEncoding { - vValSize += emptyMVCCValueHeaderSize // 17 - } + vValSize := int64(len(value.RawBytes)) + require.EqualValues(t, vValSize, 10) + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize // 17 + } - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - KeyBytes: mKeySize + vKeySize, - KeyCount: 1, - ValBytes: vValSize, - ValCount: 1, - LiveBytes: mKeySize + vKeySize + vValSize, - LiveCount: 1, - } + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + KeyBytes: mKeySize + vKeySize, + KeyCount: 1, + ValBytes: vValSize, + ValCount: 1, + LiveBytes: mKeySize + vKeySize + vValSize, + LiveCount: 1, + } - assertEq(t, engine, "after non-transactional put", aggMS, &expMS) + assertEq(t, engine, "after non-transactional put", aggMS, &expMS) - // Write a tombstone intent at t=2s. - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, - ReadTimestamp: ts2, - } - if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { - t.Fatal(err) - } + // Write a tombstone intent at t=2s. + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, + ReadTimestamp: ts2, + } + if _, err := MVCCDelete(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + t.Fatal(err) + } - mValSize := int64((&enginepb.MVCCMetadata{ - Timestamp: ts1.ToLegacyTimestamp(), - Deleted: true, - Txn: &txn.TxnMeta, - }).Size()) - require.EqualValues(t, mValSize, 46) - mValSize += 2 - - vDelSize := int64(0) // tombstone - if disableSimpleValueEncoding { - vDelSize = emptyMVCCValueHeaderSize // 7 - } + mValSize := int64((&enginepb.MVCCMetadata{ + Timestamp: ts1.ToLegacyTimestamp(), + Deleted: true, + Txn: &txn.TxnMeta, + }).Size()) + require.EqualValues(t, mValSize, 46) + mValSize += 2 + + vDelSize := int64(0) // tombstone + if disableSimpleValueEncoding { + vDelSize = emptyMVCCValueHeaderSize // 7 + } - expMS = enginepb.MVCCStats{ - LastUpdateNanos: 2e9, - KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 - KeyCount: 1, - ValBytes: mValSize + vValSize + vDelSize, // 46[+2]+10[+7] = 56[+2][+7] - ValCount: 2, - IntentCount: 1, - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vDelSize, // 12[+7] - // The original non-transactional write becomes non-live at 2s, so no age - // is accrued yet. - GCBytesAge: 0, - } - assertEq(t, engine, "after txn delete", aggMS, &expMS) - - // Now commit or abort the intent, but with a timestamp gap (i.e. this is a push-commit as it - // would happen for a SNAPSHOT txn) - - txn.WriteTimestamp.Forward(ts3) - txn.Sequence++ - - // Annoyingly, the new meta value is actually a little larger thanks to the - // sequence number. - m2ValSize := int64((&enginepb.MVCCMetadata{ - Timestamp: ts3.ToLegacyTimestamp(), - Txn: &txn.TxnMeta, - }).Size()) - - require.EqualValues(t, m2ValSize, 48) - - t.Run("Abort", func(t *testing.T) { - aggMS := *aggMS - engine := engine.NewBatch() - defer engine.Close() - - txnAbort := txn.Clone() - txnAbort.Status = roachpb.ABORTED // doesn't change m2ValSize, fortunately - if _, err := MVCCResolveWriteIntent(ctx, engine, &aggMS, - roachpb.MakeLockUpdate(txnAbort, roachpb.Span{Key: key}), - ); err != nil { - t.Fatal(err) - } + expMS = enginepb.MVCCStats{ + LastUpdateNanos: 2e9, + KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 + KeyCount: 1, + ValBytes: mValSize + vValSize + vDelSize, // 46[+2]+10[+7] = 56[+2][+7] + ValCount: 2, + IntentCount: 1, + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vDelSize, // 12[+7] + // The original non-transactional write becomes non-live at 2s, so no age + // is accrued yet. + GCBytesAge: 0, + } + assertEq(t, engine, "after txn delete", aggMS, &expMS) - expAggMS := enginepb.MVCCStats{ - LastUpdateNanos: 3e9, - KeyBytes: mKeySize + vKeySize, - KeyCount: 1, - ValBytes: vValSize, - ValCount: 1, - LiveCount: 1, - LiveBytes: mKeySize + vKeySize + vValSize, - IntentCount: 0, - IntentBytes: 0, - // The original value is visible again, so no GCBytesAge is present. Verifying this is the - // main point of this test (to prevent regression of a bug). - GCBytesAge: 0, - } - assertEq(t, engine, "after abort", &aggMS, &expAggMS) - }) - t.Run("Put", func(t *testing.T) { - aggMS := *aggMS - engine := engine.NewBatch() - defer engine.Close() - - val2 := roachpb.MakeValueFromString("longvalue") - vVal2Size := int64(len(val2.RawBytes)) - require.EqualValues(t, vVal2Size, 14) - if disableSimpleValueEncoding { - vVal2Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal2Size, 21) - } + // Now commit or abort the intent, but with a timestamp gap (i.e. this is a push-commit as it + // would happen for a SNAPSHOT txn) - txn.WriteTimestamp.Forward(ts3) - if err := MVCCPut(ctx, engine, &aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { - t.Fatal(err) - } + txn.WriteTimestamp.Forward(ts3) + txn.Sequence++ - // Annoyingly, the new meta value is actually a little larger thanks to the - // sequence number. - encDel, err := EncodeMVCCValue(MVCCValue{Value: roachpb.Value{RawBytes: []byte{}}}) - require.NoError(t, err) - mVal2SizeWithHistory := int64((&enginepb.MVCCMetadata{ - Timestamp: ts3.ToLegacyTimestamp(), - Txn: &txn.TxnMeta, - IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: encDel}, - }, - }).Size()) - expMVal2Size := 54 - if disableSimpleValueEncoding { - expMVal2Size = 61 - } - require.EqualValues(t, mVal2SizeWithHistory, expMVal2Size) - - expAggMS := enginepb.MVCCStats{ - LastUpdateNanos: 3e9, - KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 - KeyCount: 1, - ValBytes: mVal2SizeWithHistory + vValSize + vVal2Size, - ValCount: 2, - LiveCount: 1, - LiveBytes: mKeySize + mVal2SizeWithHistory + vKeySize + vVal2Size, - IntentCount: 1, - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vVal2Size, - // The original write was previously non-live at 2s because that's where the - // intent originally lived. But the intent has moved to 3s, and so has the - // moment in time at which the shadowed put became non-live; it's now 3s as - // well, so there's no contribution yet. - GCBytesAge: 0, - } - assertEq(t, engine, "after txn put", &aggMS, &expAggMS) - }) - }) - } + // Annoyingly, the new meta value is actually a little larger thanks to the + // sequence number. + m2ValSize := int64((&enginepb.MVCCMetadata{ + Timestamp: ts3.ToLegacyTimestamp(), + Txn: &txn.TxnMeta, + }).Size()) + + require.EqualValues(t, m2ValSize, 48) + + t.Run("Abort", func(t *testing.T) { + aggMS := *aggMS + engine := engine.NewBatch() + defer engine.Close() + + txnAbort := txn.Clone() + txnAbort.Status = roachpb.ABORTED // doesn't change m2ValSize, fortunately + if _, err := MVCCResolveWriteIntent(ctx, engine, &aggMS, + roachpb.MakeLockUpdate(txnAbort, roachpb.Span{Key: key}), + ); err != nil { + t.Fatal(err) + } + + expAggMS := enginepb.MVCCStats{ + LastUpdateNanos: 3e9, + KeyBytes: mKeySize + vKeySize, + KeyCount: 1, + ValBytes: vValSize, + ValCount: 1, + LiveCount: 1, + LiveBytes: mKeySize + vKeySize + vValSize, + IntentCount: 0, + IntentBytes: 0, + // The original value is visible again, so no GCBytesAge is present. Verifying this is the + // main point of this test (to prevent regression of a bug). + GCBytesAge: 0, + } + assertEq(t, engine, "after abort", &aggMS, &expAggMS) + }) + t.Run("Put", func(t *testing.T) { + aggMS := *aggMS + engine := engine.NewBatch() + defer engine.Close() + + val2 := roachpb.MakeValueFromString("longvalue") + vVal2Size := int64(len(val2.RawBytes)) + require.EqualValues(t, vVal2Size, 14) + if disableSimpleValueEncoding { + vVal2Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal2Size, 21) + } + + txn.WriteTimestamp.Forward(ts3) + if err := MVCCPut(ctx, engine, &aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { + t.Fatal(err) + } + + // Annoyingly, the new meta value is actually a little larger thanks to the + // sequence number. + encDel, err := EncodeMVCCValue(MVCCValue{Value: roachpb.Value{RawBytes: []byte{}}}) + require.NoError(t, err) + mVal2SizeWithHistory := int64((&enginepb.MVCCMetadata{ + Timestamp: ts3.ToLegacyTimestamp(), + Txn: &txn.TxnMeta, + IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ + {Sequence: 0, Value: encDel}, + }, + }).Size()) + expMVal2Size := 54 + if disableSimpleValueEncoding { + expMVal2Size = 61 + } + require.EqualValues(t, mVal2SizeWithHistory, expMVal2Size) + + expAggMS := enginepb.MVCCStats{ + LastUpdateNanos: 3e9, + KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 + KeyCount: 1, + ValBytes: mVal2SizeWithHistory + vValSize + vVal2Size, + ValCount: 2, + LiveCount: 1, + LiveBytes: mKeySize + mVal2SizeWithHistory + vKeySize + vVal2Size, + IntentCount: 1, + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vVal2Size, + // The original write was previously non-live at 2s because that's where the + // intent originally lived. But the intent has moved to 3s, and so has the + // moment in time at which the shadowed put became non-live; it's now 3s as + // well, so there's no contribution yet. + GCBytesAge: 0, + } + assertEq(t, engine, "after txn put", &aggMS, &expAggMS) + }) } // TestMVCCStatsDelDelGC prevents regression of a bug in MVCCGarbageCollect @@ -970,70 +943,66 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { func TestMVCCStatsDelDelGC(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + engine := NewDefaultInMemForTesting() + defer engine.Close() - assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} + assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - // Write tombstones at ts1 and ts2. - if _, err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } - if _, err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } + key := roachpb.Key("a") + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2e9} - mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - vKeySize := MVCCVersionTimestampSize // 12 - vValSize := int64(0) // tombstone - if disableSimpleValueEncoding { - vValSize = emptyMVCCValueHeaderSize // 7 - } + // Write tombstones at ts1 and ts2. + if _, err := MVCCDelete(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } + if _, err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 2e9, - KeyBytes: mKeySize + 2*vKeySize, // 26 - ValBytes: 2 * vValSize, - KeyCount: 1, - ValCount: 2, - GCBytesAge: 1 * (vKeySize + vValSize), // first tombstone, aged from ts1 to ts2 - } - assertEq(t, engine, "after two puts", aggMS, &expMS) - - // Run a GC invocation that clears it all. There used to be a bug here when - // we allowed limiting the number of deleted keys. Passing zero (i.e. remove - // one key and then bail) would mess up the stats, since the implementation - // would assume that the (implicit or explicit) meta entry was going to be - // removed, but this is only true when all values actually go away. - if err := MVCCGarbageCollect( - ctx, - engine, - aggMS, - []roachpb.GCRequest_GCKey{{ - Key: key, - Timestamp: ts2, - }}, - ts2, - ); err != nil { - t.Fatal(err) - } + mKeySize := int64(mvccKey(key).EncodedSize()) // 2 + vKeySize := MVCCVersionTimestampSize // 12 + vValSize := int64(0) // tombstone + if disableSimpleValueEncoding { + vValSize = emptyMVCCValueHeaderSize // 7 + } - expAggMS := enginepb.MVCCStats{ - LastUpdateNanos: 2e9, - } + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 2e9, + KeyBytes: mKeySize + 2*vKeySize, // 26 + ValBytes: 2 * vValSize, + KeyCount: 1, + ValCount: 2, + GCBytesAge: 1 * (vKeySize + vValSize), // first tombstone, aged from ts1 to ts2 + } + assertEq(t, engine, "after two puts", aggMS, &expMS) + + // Run a GC invocation that clears it all. There used to be a bug here when + // we allowed limiting the number of deleted keys. Passing zero (i.e. remove + // one key and then bail) would mess up the stats, since the implementation + // would assume that the (implicit or explicit) meta entry was going to be + // removed, but this is only true when all values actually go away. + if err := MVCCGarbageCollect( + ctx, + engine, + aggMS, + []roachpb.GCRequest_GCKey{{ + Key: key, + Timestamp: ts2, + }}, + ts2, + ); err != nil { + t.Fatal(err) + } - assertEq(t, engine, "after GC", aggMS, &expAggMS) - }) + expAggMS := enginepb.MVCCStats{ + LastUpdateNanos: 2e9, } + + assertEq(t, engine, "after GC", aggMS, &expAggMS) } // TestMVCCStatsPutIntentTimestampNotPutTimestamp exercises a scenario in which @@ -1053,99 +1022,95 @@ func TestMVCCStatsDelDelGC(t *testing.T) { func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} - - assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - - key := roachpb.Key("a") - ts201 := hlc.Timestamp{WallTime: 2e9 + 1} - ts099 := hlc.Timestamp{WallTime: 1e9 - 1} - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts201}, - ReadTimestamp: ts099, - } - // Write an intent at 2s+1. - value := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { - t.Fatal(err) - } + engine := NewDefaultInMemForTesting() + defer engine.Close() - mKeySize := int64(mvccKey(key).EncodedSize()) // 2 - m1ValSize := int64((&enginepb.MVCCMetadata{ // 46 - Timestamp: ts201.ToLegacyTimestamp(), - Txn: &txn.TxnMeta, - }).Size()) - m1ValSize += 2 - vKeySize := MVCCVersionTimestampSize // 12 - vValSize := int64(len(value.RawBytes)) // 10 - if disableSimpleValueEncoding { - vValSize += emptyMVCCValueHeaderSize // 17 - } + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 2e9 + 1, - LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 68[+2][+7] - LiveCount: 1, - KeyBytes: mKeySize + vKeySize, // 14 - KeyCount: 1, - ValBytes: m1ValSize + vValSize, // (46[+2])+(10[+7]) = 54[+2][+7] - ValCount: 1, - IntentCount: 1, - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] - } - assertEq(t, engine, "after first put", aggMS, &expMS) - - // Replace the intent with an identical one, but we write it at 1s-1 now. If - // you're confused, don't worry. There are two timestamps here: the one in - // the txn (which is, perhaps surprisingly, only really used when - // committing/aborting intents), and the timestamp passed directly to - // MVCCPut (which is where the intent will actually end up being written at, - // and which usually corresponds to txn.ReadTimestamp). - txn.Sequence++ - txn.WriteTimestamp = ts099 - - // Annoyingly, the new meta value is actually a little larger thanks to the - // sequence number. - encValue, err := EncodeMVCCValue(MVCCValue{Value: value}) - require.NoError(t, err) - m2ValSize := int64((&enginepb.MVCCMetadata{ // 46 - Timestamp: ts201.ToLegacyTimestamp(), - Txn: &txn.TxnMeta, - IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: encValue}, - }, - }).Size()) - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { - t.Fatal(err) - } + assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - expAggMS := enginepb.MVCCStats{ - // Even though we tried to put a new intent at an older timestamp, it - // will have been written at 2E9+1, so the age will be 0. - IntentAge: 0, - - LastUpdateNanos: 2e9 + 1, - LiveBytes: mKeySize + m2ValSize + vKeySize + vValSize, // 2+(46[+7])+12+(10[+7]) = 70[+14] - LiveCount: 1, - KeyBytes: mKeySize + vKeySize, // 14 - KeyCount: 1, - ValBytes: m2ValSize + vValSize, // (46[+7])+(10[+7]) = 56[+14] - ValCount: 1, - IntentCount: 1, - SeparatedIntentCount: 1, - IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] - } + key := roachpb.Key("a") + ts201 := hlc.Timestamp{WallTime: 2e9 + 1} + ts099 := hlc.Timestamp{WallTime: 1e9 - 1} + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts201}, + ReadTimestamp: ts099, + } + // Write an intent at 2s+1. + value := roachpb.MakeValueFromString("value") + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { + t.Fatal(err) + } - assertEq(t, engine, "after second put", aggMS, &expAggMS) - }) + mKeySize := int64(mvccKey(key).EncodedSize()) // 2 + m1ValSize := int64((&enginepb.MVCCMetadata{ // 46 + Timestamp: ts201.ToLegacyTimestamp(), + Txn: &txn.TxnMeta, + }).Size()) + m1ValSize += 2 + vKeySize := MVCCVersionTimestampSize // 12 + vValSize := int64(len(value.RawBytes)) // 10 + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize // 17 + } + + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 2e9 + 1, + LiveBytes: mKeySize + m1ValSize + vKeySize + vValSize, // 2+(46[+2])+12+(10[+7]) = 68[+2][+7] + LiveCount: 1, + KeyBytes: mKeySize + vKeySize, // 14 + KeyCount: 1, + ValBytes: m1ValSize + vValSize, // (46[+2])+(10[+7]) = 54[+2][+7] + ValCount: 1, + IntentCount: 1, + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] } + assertEq(t, engine, "after first put", aggMS, &expMS) + + // Replace the intent with an identical one, but we write it at 1s-1 now. If + // you're confused, don't worry. There are two timestamps here: the one in + // the txn (which is, perhaps surprisingly, only really used when + // committing/aborting intents), and the timestamp passed directly to + // MVCCPut (which is where the intent will actually end up being written at, + // and which usually corresponds to txn.ReadTimestamp). + txn.Sequence++ + txn.WriteTimestamp = ts099 + + // Annoyingly, the new meta value is actually a little larger thanks to the + // sequence number. + encValue, err := EncodeMVCCValue(MVCCValue{Value: value}) + require.NoError(t, err) + m2ValSize := int64((&enginepb.MVCCMetadata{ // 46 + Timestamp: ts201.ToLegacyTimestamp(), + Txn: &txn.TxnMeta, + IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ + {Sequence: 0, Value: encValue}, + }, + }).Size()) + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, value, txn); err != nil { + t.Fatal(err) + } + + expAggMS := enginepb.MVCCStats{ + // Even though we tried to put a new intent at an older timestamp, it + // will have been written at 2E9+1, so the age will be 0. + IntentAge: 0, + + LastUpdateNanos: 2e9 + 1, + LiveBytes: mKeySize + m2ValSize + vKeySize + vValSize, // 2+(46[+7])+12+(10[+7]) = 70[+14] + LiveCount: 1, + KeyBytes: mKeySize + vKeySize, // 14 + KeyCount: 1, + ValBytes: m2ValSize + vValSize, // (46[+7])+(10[+7]) = 56[+14] + ValCount: 1, + IntentCount: 1, + SeparatedIntentCount: 1, + IntentBytes: vKeySize + vValSize, // 12+(10[+7]) = 22[+7] + } + + assertEq(t, engine, "after second put", aggMS, &expAggMS) } // TestMVCCStatsPutWaitDeleteGC puts a value, deletes it, and runs a GC that @@ -1153,96 +1118,93 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + engine := NewDefaultInMemForTesting() + defer engine.Close() - assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - key := roachpb.Key("a") + assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} + key := roachpb.Key("a") - // Write a value at ts1. - val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { - t.Fatal(err) - } + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2e9} - mKeySize := int64(mvccKey(key).EncodedSize()) - require.EqualValues(t, mKeySize, 2) + // Write a value at ts1. + val1 := roachpb.MakeValueFromString("value") + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { + t.Fatal(err) + } - vKeySize := MVCCVersionTimestampSize - require.EqualValues(t, vKeySize, 12) + mKeySize := int64(mvccKey(key).EncodedSize()) + require.EqualValues(t, mKeySize, 2) - vValSize := int64(len(val1.RawBytes)) - require.EqualValues(t, vValSize, 10) - if disableSimpleValueEncoding { - vValSize += emptyMVCCValueHeaderSize - require.EqualValues(t, vValSize, 17) - } + vKeySize := MVCCVersionTimestampSize + require.EqualValues(t, vKeySize, 12) - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - KeyCount: 1, - KeyBytes: mKeySize + vKeySize, // 2+12 = 14 - ValCount: 1, - ValBytes: vValSize, // 10[+7] - LiveCount: 1, - LiveBytes: mKeySize + vKeySize + vValSize, // 2+12+(10[+7]) = 24[+7] - } - assertEq(t, engine, "after first put", aggMS, &expMS) + vValSize := int64(len(val1.RawBytes)) + require.EqualValues(t, vValSize, 10) + if disableSimpleValueEncoding { + vValSize += emptyMVCCValueHeaderSize + require.EqualValues(t, vValSize, 17) + } - // Delete the value at ts5. + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + KeyCount: 1, + KeyBytes: mKeySize + vKeySize, // 2+12 = 14 + ValCount: 1, + ValBytes: vValSize, // 10[+7] + LiveCount: 1, + LiveBytes: mKeySize + vKeySize + vValSize, // 2+12+(10[+7]) = 24[+7] + } + assertEq(t, engine, "after first put", aggMS, &expMS) - if _, err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } + // Delete the value at ts5. - vVal2Size := int64(0) // tombstone - if disableSimpleValueEncoding { - vVal2Size = emptyMVCCValueHeaderSize // 7 - } + if _, err := MVCCDelete(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } - expMS = enginepb.MVCCStats{ - LastUpdateNanos: 2e9, - KeyCount: 1, - KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 - ValBytes: vValSize + vVal2Size, // 10[+7] - ValCount: 2, - LiveBytes: 0, - LiveCount: 0, - GCBytesAge: 0, // before a fix, this was vKeySize + vValSize - } + vVal2Size := int64(0) // tombstone + if disableSimpleValueEncoding { + vVal2Size = emptyMVCCValueHeaderSize // 7 + } - assertEq(t, engine, "after delete", aggMS, &expMS) + expMS = enginepb.MVCCStats{ + LastUpdateNanos: 2e9, + KeyCount: 1, + KeyBytes: mKeySize + 2*vKeySize, // 2+2*12 = 26 + ValBytes: vValSize + vVal2Size, // 10[+7] + ValCount: 2, + LiveBytes: 0, + LiveCount: 0, + GCBytesAge: 0, // before a fix, this was vKeySize + vValSize + } - if err := MVCCGarbageCollect(ctx, engine, aggMS, []roachpb.GCRequest_GCKey{{ - Key: key, - Timestamp: ts1, - }}, ts2); err != nil { - t.Fatal(err) - } + assertEq(t, engine, "after delete", aggMS, &expMS) - expMS = enginepb.MVCCStats{ - LastUpdateNanos: 2e9, - KeyCount: 1, - KeyBytes: mKeySize + vKeySize, // 2+12 = 14 - ValBytes: vVal2Size, - ValCount: 1, - LiveBytes: 0, - LiveCount: 0, - GCBytesAge: 0, // before a fix, this was vKeySize + vValSize - } + if err := MVCCGarbageCollect(ctx, engine, aggMS, []roachpb.GCRequest_GCKey{{ + Key: key, + Timestamp: ts1, + }}, ts2); err != nil { + t.Fatal(err) + } - assertEq(t, engine, "after GC", aggMS, &expMS) - }) + expMS = enginepb.MVCCStats{ + LastUpdateNanos: 2e9, + KeyCount: 1, + KeyBytes: mKeySize + vKeySize, // 2+12 = 14 + ValBytes: vVal2Size, + ValCount: 1, + LiveBytes: 0, + LiveCount: 0, + GCBytesAge: 0, // before a fix, this was vKeySize + vValSize } + + assertEq(t, engine, "after GC", aggMS, &expMS) } // TestMVCCStatsSysTxnPutPut prevents regression of a bug that, when rewriting an intent @@ -1250,104 +1212,101 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { func TestMVCCStatsTxnSysPutPut(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + engine := NewDefaultInMemForTesting() + defer engine.Close() - assertEqLocal(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - key := keys.RangeDescriptorKey(roachpb.RKey("a")) + assertEqLocal(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} + key := keys.RangeDescriptorKey(roachpb.RKey("a")) - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, - ReadTimestamp: ts1, - } + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2e9} - // Write an intent at ts1. - val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val1, txn); err != nil { - t.Fatal(err) - } + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, + ReadTimestamp: ts1, + } - mKeySize := int64(mvccKey(key).EncodedSize()) - require.EqualValues(t, mKeySize, 11) - - mValSize := int64((&enginepb.MVCCMetadata{ - Timestamp: ts1.ToLegacyTimestamp(), - Deleted: false, - Txn: &txn.TxnMeta, - }).Size()) - require.EqualValues(t, mValSize, 46) - mValSize += 2 - - vKeySize := MVCCVersionTimestampSize - require.EqualValues(t, vKeySize, 12) - - vVal1Size := int64(len(val1.RawBytes)) - require.EqualValues(t, vVal1Size, 10) - if disableSimpleValueEncoding { - vVal1Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal1Size, 17) - } + // Write an intent at ts1. + val1 := roachpb.MakeValueFromString("value") + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val1, txn); err != nil { + t.Fatal(err) + } - val2 := roachpb.MakeValueFromString("longvalue") - vVal2Size := int64(len(val2.RawBytes)) - require.EqualValues(t, vVal2Size, 14) - if disableSimpleValueEncoding { - vVal2Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal2Size, 21) - } + mKeySize := int64(mvccKey(key).EncodedSize()) + require.EqualValues(t, mKeySize, 11) + + mValSize := int64((&enginepb.MVCCMetadata{ + Timestamp: ts1.ToLegacyTimestamp(), + Deleted: false, + Txn: &txn.TxnMeta, + }).Size()) + require.EqualValues(t, mValSize, 46) + mValSize += 2 + + vKeySize := MVCCVersionTimestampSize + require.EqualValues(t, vKeySize, 12) + + vVal1Size := int64(len(val1.RawBytes)) + require.EqualValues(t, vVal1Size, 10) + if disableSimpleValueEncoding { + vVal1Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal1Size, 17) + } - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - SysBytes: mKeySize + mValSize + vKeySize + vVal1Size, // 11+(46[+2])+12+(10[+7]) = 79[+2][+7] - SysCount: 1, - } - assertEqLocal(t, engine, "after first put", aggMS, &expMS) - - // Rewrite the intent to ts2 with a different value. - txn.WriteTimestamp.Forward(ts2) - txn.Sequence++ - - // The new meta value grows because we've bumped `txn.Sequence`. - // The value also grows as the older value is part of the same - // transaction and so contributes to the intent history. - encVal1, err := EncodeMVCCValue(MVCCValue{Value: val1}) - require.NoError(t, err) - mVal2Size := int64((&enginepb.MVCCMetadata{ - Timestamp: ts2.ToLegacyTimestamp(), - Deleted: false, - Txn: &txn.TxnMeta, - IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ - {Sequence: 0, Value: encVal1}, - }, - }).Size()) - expMVal2Size := 64 - if disableSimpleValueEncoding { - expMVal2Size = 71 - } - require.EqualValues(t, mVal2Size, expMVal2Size) + val2 := roachpb.MakeValueFromString("longvalue") + vVal2Size := int64(len(val2.RawBytes)) + require.EqualValues(t, vVal2Size, 14) + if disableSimpleValueEncoding { + vVal2Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal2Size, 21) + } - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { - t.Fatal(err) - } + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + SysBytes: mKeySize + mValSize + vKeySize + vVal1Size, // 11+(46[+2])+12+(10[+7]) = 79[+2][+7] + SysCount: 1, + } + assertEqLocal(t, engine, "after first put", aggMS, &expMS) + + // Rewrite the intent to ts2 with a different value. + txn.WriteTimestamp.Forward(ts2) + txn.Sequence++ + + // The new meta value grows because we've bumped `txn.Sequence`. + // The value also grows as the older value is part of the same + // transaction and so contributes to the intent history. + encVal1, err := EncodeMVCCValue(MVCCValue{Value: val1}) + require.NoError(t, err) + mVal2Size := int64((&enginepb.MVCCMetadata{ + Timestamp: ts2.ToLegacyTimestamp(), + Deleted: false, + Txn: &txn.TxnMeta, + IntentHistory: []enginepb.MVCCMetadata_SequencedIntent{ + {Sequence: 0, Value: encVal1}, + }, + }).Size()) + expMVal2Size := 64 + if disableSimpleValueEncoding { + expMVal2Size = 71 + } + require.EqualValues(t, mVal2Size, expMVal2Size) - expMS = enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - SysBytes: mKeySize + mVal2Size + vKeySize + vVal2Size, // 11+(46[+7])+12+14 = 83[+7] - SysCount: 1, - } + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val2, txn); err != nil { + t.Fatal(err) + } - assertEqLocal(t, engine, "after intent rewrite", aggMS, &expMS) - }) + expMS = enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + SysBytes: mKeySize + mVal2Size + vKeySize + vVal2Size, // 11+(46[+7])+12+14 = 83[+7] + SysCount: 1, } + + assertEqLocal(t, engine, "after intent rewrite", aggMS, &expMS) } // TestMVCCStatsTxnSysPutAbort prevents regression of a bug that, when aborting @@ -1356,80 +1315,76 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { func TestMVCCStatsTxnSysPutAbort(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - assertEqLocal(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + assertEqLocal(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - key := keys.RangeDescriptorKey(roachpb.RKey("a")) + key := keys.RangeDescriptorKey(roachpb.RKey("a")) - ts1 := hlc.Timestamp{WallTime: 1e9} - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, - ReadTimestamp: ts1, - } + ts1 := hlc.Timestamp{WallTime: 1e9} + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, + ReadTimestamp: ts1, + } - // Write a system intent at ts1. - val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val1, txn); err != nil { - t.Fatal(err) - } + // Write a system intent at ts1. + val1 := roachpb.MakeValueFromString("value") + if err := MVCCPut(ctx, engine, aggMS, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, val1, txn); err != nil { + t.Fatal(err) + } - mKeySize := int64(mvccKey(key).EncodedSize()) - require.EqualValues(t, mKeySize, 11) - - mValSize := int64((&enginepb.MVCCMetadata{ - Timestamp: ts1.ToLegacyTimestamp(), - Deleted: false, - Txn: &txn.TxnMeta, - }).Size()) - require.EqualValues(t, mValSize, 46) - mValSize += 2 - - vKeySize := MVCCVersionTimestampSize - require.EqualValues(t, vKeySize, 12) - - vVal1Size := int64(len(val1.RawBytes)) - require.EqualValues(t, vVal1Size, 10) - if disableSimpleValueEncoding { - vVal1Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal1Size, 17) - } + mKeySize := int64(mvccKey(key).EncodedSize()) + require.EqualValues(t, mKeySize, 11) + + mValSize := int64((&enginepb.MVCCMetadata{ + Timestamp: ts1.ToLegacyTimestamp(), + Deleted: false, + Txn: &txn.TxnMeta, + }).Size()) + require.EqualValues(t, mValSize, 46) + mValSize += 2 + + vKeySize := MVCCVersionTimestampSize + require.EqualValues(t, vKeySize, 12) + + vVal1Size := int64(len(val1.RawBytes)) + require.EqualValues(t, vVal1Size, 10) + if disableSimpleValueEncoding { + vVal1Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal1Size, 17) + } - val2 := roachpb.MakeValueFromString("longvalue") - vVal2Size := int64(len(val2.RawBytes)) - require.EqualValues(t, vVal2Size, 14) - if disableSimpleValueEncoding { - vVal2Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal2Size, 21) - } + val2 := roachpb.MakeValueFromString("longvalue") + vVal2Size := int64(len(val2.RawBytes)) + require.EqualValues(t, vVal2Size, 14) + if disableSimpleValueEncoding { + vVal2Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal2Size, 21) + } - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - SysBytes: mKeySize + mValSize + vKeySize + vVal1Size, // 11+(46[+2])+12+(10[+7]) = 79[+2][+7] - SysCount: 1, - } - assertEqLocal(t, engine, "after first put", aggMS, &expMS) - - // Now abort the intent. - txn.Status = roachpb.ABORTED - if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, - roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}), - ); err != nil { - t.Fatal(err) - } + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + SysBytes: mKeySize + mValSize + vKeySize + vVal1Size, // 11+(46[+2])+12+(10[+7]) = 79[+2][+7] + SysCount: 1, + } + assertEqLocal(t, engine, "after first put", aggMS, &expMS) - expMS = enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - } - assertEqLocal(t, engine, "after aborting", aggMS, &expMS) - }) + // Now abort the intent. + txn.Status = roachpb.ABORTED + if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, + roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}), + ); err != nil { + t.Fatal(err) + } + + expMS = enginepb.MVCCStats{ + LastUpdateNanos: 1e9, } + assertEqLocal(t, engine, "after aborting", aggMS, &expMS) } // TestMVCCStatsSysPutPut prevents regression of a bug that, when writing a new @@ -1437,70 +1392,66 @@ func TestMVCCStatsTxnSysPutAbort(t *testing.T) { func TestMVCCStatsSysPutPut(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - ctx := context.Background() - aggMS := &enginepb.MVCCStats{} + engine := NewDefaultInMemForTesting() + defer engine.Close() - assertEqLocal(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) + ctx := context.Background() + aggMS := &enginepb.MVCCStats{} - key := keys.RangeDescriptorKey(roachpb.RKey("a")) + assertEqLocal(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} + key := keys.RangeDescriptorKey(roachpb.RKey("a")) - // Write a value at ts1. - val1 := roachpb.MakeValueFromString("value") - if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { - t.Fatal(err) - } + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2e9} - mKeySize := int64(mvccKey(key).EncodedSize()) - require.EqualValues(t, mKeySize, 11) + // Write a value at ts1. + val1 := roachpb.MakeValueFromString("value") + if err := MVCCPut(ctx, engine, aggMS, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { + t.Fatal(err) + } - vKeySize := MVCCVersionTimestampSize - require.EqualValues(t, vKeySize, 12) + mKeySize := int64(mvccKey(key).EncodedSize()) + require.EqualValues(t, mKeySize, 11) - vVal1Size := int64(len(val1.RawBytes)) - require.EqualValues(t, vVal1Size, 10) - if disableSimpleValueEncoding { - vVal1Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal1Size, 17) - } + vKeySize := MVCCVersionTimestampSize + require.EqualValues(t, vKeySize, 12) - val2 := roachpb.MakeValueFromString("longvalue") - vVal2Size := int64(len(val2.RawBytes)) - require.EqualValues(t, vVal2Size, 14) - if disableSimpleValueEncoding { - vVal2Size += emptyMVCCValueHeaderSize - require.EqualValues(t, vVal2Size, 21) - } + vVal1Size := int64(len(val1.RawBytes)) + require.EqualValues(t, vVal1Size, 10) + if disableSimpleValueEncoding { + vVal1Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal1Size, 17) + } - expMS := enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - SysBytes: mKeySize + vKeySize + vVal1Size, // 11+12+(10[+7]) = 33[+7] - SysCount: 1, - } - assertEqLocal(t, engine, "after first put", aggMS, &expMS) + val2 := roachpb.MakeValueFromString("longvalue") + vVal2Size := int64(len(val2.RawBytes)) + require.EqualValues(t, vVal2Size, 14) + if disableSimpleValueEncoding { + vVal2Size += emptyMVCCValueHeaderSize + require.EqualValues(t, vVal2Size, 21) + } - // Put another value at ts2. + expMS := enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + SysBytes: mKeySize + vKeySize + vVal1Size, // 11+12+(10[+7]) = 33[+7] + SysCount: 1, + } + assertEqLocal(t, engine, "after first put", aggMS, &expMS) - if err := MVCCPut(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, val2, nil); err != nil { - t.Fatal(err) - } + // Put another value at ts2. - expMS = enginepb.MVCCStats{ - LastUpdateNanos: 1e9, - SysBytes: mKeySize + 2*vKeySize + vVal1Size + vVal2Size, - SysCount: 1, - } + if err := MVCCPut(ctx, engine, aggMS, key, ts2, hlc.ClockTimestamp{}, val2, nil); err != nil { + t.Fatal(err) + } - assertEqLocal(t, engine, "after second put", aggMS, &expMS) - }) + expMS = enginepb.MVCCStats{ + LastUpdateNanos: 1e9, + SysBytes: mKeySize + 2*vKeySize + vVal1Size + vVal2Size, + SysCount: 1, } + + assertEqLocal(t, engine, "after second put", aggMS, &expMS) } var mvccStatsTests = []struct { diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 1eec04e636e6..23a24529cf85 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -83,17 +83,6 @@ func createTestPebbleEngine(opts ...ConfigOption) Engine { return NewDefaultInMemForTesting(opts...) } -// TODO(sumeer): the following is legacy from when we had multiple engine -// implementations. Some tests are switched over to only create Pebble, since -// the create method does not provide control over cluster.Settings. Switch -// the rest and remove this. -var mvccEngineImpls = []struct { - name string - create func(opts ...ConfigOption) Engine -}{ - {"pebble", createTestPebbleEngine}, -} - // makeTxn creates a new transaction using the specified base // txn and timestamp. func makeTxn(baseTxn roachpb.Transaction, ts hlc.Timestamp) *roachpb.Transaction { @@ -243,20 +232,16 @@ func TestMVCCGetNotExist(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - value, _, err := MVCCGet(context.Background(), engine, testKey1, hlc.Timestamp{Logical: 1}, - MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if value != nil { - t.Fatal("the value should be empty") - } - }) + value, _, err := MVCCGet(context.Background(), engine, testKey1, hlc.Timestamp{Logical: 1}, + MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if value != nil { + t.Fatal("the value should be empty") } } @@ -265,39 +250,31 @@ func TestMVCCGetNoMoreOldVersion(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - // Need to handle the case here where the scan takes us to the - // next key, which may not match the key we're looking for. In - // other words, if we're looking for a, and we have the - // following keys: - // - // a: MVCCMetadata(a) - // a - // b: MVCCMetadata(b) - // b - // - // If we search for a, the scan should not return "b". - - engine := engineImpl.create() - defer engine.Close() - - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if value != nil { - t.Fatal("the value should be empty") - } - }) + engine := NewDefaultInMemForTesting() + defer engine.Close() + // Need to handle the case here where the scan takes us to the + // next key, which may not match the key we're looking for. In + // other words, if we're looking for a, and we have the + // following keys: + // + // a: MVCCMetadata(a) + // a + // b: MVCCMetadata(b) + // b + // + // If we search for a, the scan should not return "b". + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if value != nil { + t.Fatal("the value should be empty") } } @@ -306,57 +283,52 @@ func TestMVCCGetAndDelete(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() + engine := NewDefaultInMemForTesting() + defer engine.Close() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if value == nil { - t.Fatal("the value should not be empty") - } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if value == nil { + t.Fatal("the value should not be empty") + } - _, err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil) - if err != nil { - t.Fatal(err) - } + _, err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil) + if err != nil { + t.Fatal(err) + } - // Read the latest version which should be deleted. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if value != nil { - t.Fatal("the value should be empty") - } - // Read the latest version with tombstone. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, - MVCCGetOptions{Tombstones: true}) - if err != nil { - t.Fatal(err) - } else if value == nil || len(value.RawBytes) != 0 { - t.Fatalf("the value should be non-nil with empty RawBytes; got %+v", value) - } + // Read the latest version which should be deleted. + value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if value != nil { + t.Fatal("the value should be empty") + } + // Read the latest version with tombstone. + value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, + MVCCGetOptions{Tombstones: true}) + if err != nil { + t.Fatal(err) + } else if value == nil || len(value.RawBytes) != 0 { + t.Fatalf("the value should be non-nil with empty RawBytes; got %+v", value) + } - // Read the old version which should still exist. - for _, logical := range []int32{0, math.MaxInt32} { - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2, Logical: logical}, - MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if value == nil { - t.Fatal("the value should not be empty") - } - } - }) + // Read the old version which should still exist. + for _, logical := range []int32{0, math.MaxInt32} { + value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2, Logical: logical}, + MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if value == nil { + t.Fatal("the value should not be empty") + } } } @@ -367,48 +339,44 @@ func TestMVCCGetAndDelete(t *testing.T) { func TestMVCCWriteWithOlderTimestampAfterDeletionOfNonexistentKey(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if _, err := MVCCDelete(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } + if _, err := MVCCDelete(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } - if err := MVCCPut(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); !testutils.IsError( - err, "write for key \"/db1\" at timestamp 0.000000001,0 too old; wrote at 0.000000003,1", - ) { - t.Fatal(err) - } + if err := MVCCPut(context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); !testutils.IsError( + err, "write for key \"/db1\" at timestamp 0.000000001,0 too old; wrote at 0.000000003,1", + ) { + t.Fatal(err) + } - value, _, err := MVCCGet(context.Background(), engine, testKey1, hlc.Timestamp{WallTime: 2}, - MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - // The attempted write at ts(1,0) was performed at ts(3,1), so we should - // not see it at ts(2,0). - if value != nil { - t.Fatalf("value present at TS = %s", value.Timestamp) - } + value, _, err := MVCCGet(context.Background(), engine, testKey1, hlc.Timestamp{WallTime: 2}, + MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + // The attempted write at ts(1,0) was performed at ts(3,1), so we should + // not see it at ts(2,0). + if value != nil { + t.Fatalf("value present at TS = %s", value.Timestamp) + } - // Read the latest version which will be the value written with the timestamp pushed. - value, _, err = MVCCGet(context.Background(), engine, testKey1, hlc.Timestamp{WallTime: 4}, - MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if value == nil { - t.Fatal("value doesn't exist") - } - if !bytes.Equal(value.RawBytes, value1.RawBytes) { - t.Errorf("expected %q; got %q", value1.RawBytes, value.RawBytes) - } - if expTS := (hlc.Timestamp{WallTime: 3, Logical: 1}); value.Timestamp != expTS { - t.Fatalf("timestamp was not pushed: %s, expected %s", value.Timestamp, expTS) - } - }) + // Read the latest version which will be the value written with the timestamp pushed. + value, _, err = MVCCGet(context.Background(), engine, testKey1, hlc.Timestamp{WallTime: 4}, + MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if value == nil { + t.Fatal("value doesn't exist") + } + if !bytes.Equal(value.RawBytes, value1.RawBytes) { + t.Errorf("expected %q; got %q", value1.RawBytes, value.RawBytes) + } + if expTS := (hlc.Timestamp{WallTime: 3, Logical: 1}); value.Timestamp != expTS { + t.Fatalf("timestamp was not pushed: %s, expected %s", value.Timestamp, expTS) } } @@ -417,39 +385,35 @@ func TestMVCCInlineWithTxn(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Put an inline value. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } + // Put an inline value. + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } - // Now verify inline get. - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !reflect.DeepEqual(value1, *value) { - t.Errorf("the inline value should be %v; got %v", value1, *value) - } + // Now verify inline get. + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(value1, *value) { + t.Errorf("the inline value should be %v; got %v", value1, *value) + } - // Verify inline get with txn does still work (this will happen on a - // scan if the distributed sender is forced to wrap it in a txn). - if _, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{}, MVCCGetOptions{ - Txn: txn1, - }); err != nil { - t.Error(err) - } + // Verify inline get with txn does still work (this will happen on a + // scan if the distributed sender is forced to wrap it in a txn). + if _, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{}, MVCCGetOptions{ + Txn: txn1, + }); err != nil { + t.Error(err) + } - // Verify inline put with txn is an error. - err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{}, hlc.ClockTimestamp{}, value2, txn2) - if !testutils.IsError(err, "writes not allowed within transactions") { - t.Errorf("unexpected error: %+v", err) - } - }) + // Verify inline put with txn is an error. + err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{}, hlc.ClockTimestamp{}, value2, txn2) + if !testutils.IsError(err, "writes not allowed within transactions") { + t.Errorf("unexpected error: %+v", err) } } @@ -460,17 +424,13 @@ func TestMVCCDeleteMissingKey(t *testing.T) { DisableMetamorphicSimpleValueEncoding(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if _, err := MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } - require.Empty(t, mvccGetRaw(t, engine, mvccKey(testKey1))) - }) + if _, err := MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) } + require.Empty(t, mvccGetRaw(t, engine, mvccKey(testKey1))) } func TestMVCCGetAndDeleteInTxn(t *testing.T) { @@ -479,57 +439,53 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn); err != nil { - t.Fatal(err) - } + txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn.Sequence++ + if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn); err != nil { + t.Fatal(err) + } - if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ - Txn: txn, - }); err != nil { - t.Fatal(err) - } else if value == nil { - t.Fatal("the value should not be empty") - } + if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ + Txn: txn, + }); err != nil { + t.Fatal(err) + } else if value == nil { + t.Fatal("the value should not be empty") + } - txn.Sequence++ - txn.WriteTimestamp = hlc.Timestamp{WallTime: 3} - if _, err := MVCCDelete(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { - t.Fatal(err) - } + txn.Sequence++ + txn.WriteTimestamp = hlc.Timestamp{WallTime: 3} + if _, err := MVCCDelete(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + t.Fatal(err) + } - // Read the latest version which should be deleted. - if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{ - Txn: txn, - }); err != nil { - t.Fatal(err) - } else if value != nil { - t.Fatal("the value should be empty") - } - // Read the latest version with tombstone. - if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{ - Tombstones: true, - Txn: txn, - }); err != nil { - t.Fatal(err) - } else if value == nil || len(value.RawBytes) != 0 { - t.Fatalf("the value should be non-nil with empty RawBytes; got %+v", value) - } + // Read the latest version which should be deleted. + if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{ + Txn: txn, + }); err != nil { + t.Fatal(err) + } else if value != nil { + t.Fatal("the value should be empty") + } + // Read the latest version with tombstone. + if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{ + Tombstones: true, + Txn: txn, + }); err != nil { + t.Fatal(err) + } else if value == nil || len(value.RawBytes) != 0 { + t.Fatalf("the value should be non-nil with empty RawBytes; got %+v", value) + } - // Read the old version which shouldn't exist, as within a - // transaction, we delete previous values. - if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}); err != nil { - t.Fatal(err) - } else if value != nil { - t.Fatalf("expected value nil, got: %s", value) - } - }) + // Read the old version which shouldn't exist, as within a + // transaction, we delete previous values. + if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}); err != nil { + t.Fatal(err) + } else if value != nil { + t.Fatalf("expected value nil, got: %s", value) } } @@ -538,26 +494,21 @@ func TestMVCCGetWriteIntentError(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() + engine := NewDefaultInMemForTesting() + defer engine.Close() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + t.Fatal(err) + } - if _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}); err == nil { - t.Fatal("cannot read the value of a write intent without TxnID") - } + if _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}); err == nil { + t.Fatal("cannot read the value of a write intent without TxnID") + } - if _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ - Txn: txn2, - }); err == nil { - t.Fatal("cannot read the value of a write intent from a different TxnID") - } - }) + if _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + Txn: txn2, + }); err == nil { + t.Fatal("cannot read the value of a write intent from a different TxnID") } } @@ -572,123 +523,119 @@ func TestMVCCScanWriteIntentError(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - ts := []hlc.Timestamp{{Logical: 1}, {Logical: 2}, {Logical: 3}, {Logical: 4}, {Logical: 5}, {Logical: 6}, {Logical: 7}} + ts := []hlc.Timestamp{{Logical: 1}, {Logical: 2}, {Logical: 3}, {Logical: 4}, {Logical: 5}, {Logical: 6}, {Logical: 7}} + + txn1ts := makeTxn(*txn1, ts[2]) + txn2ts := makeTxn(*txn2, ts[5]) + txnMap := map[int]*roachpb.Transaction{ + 2: txn1ts, + 5: txn2ts, + 6: txn2ts, + 7: txn2ts, + } + + fixtureKVs := []roachpb.KeyValue{ + {Key: testKey1, Value: mkVal("testValue1 pre", ts[0])}, + {Key: testKey4, Value: mkVal("testValue4 pre", ts[1])}, + {Key: testKey1, Value: mkVal("testValue1", ts[2])}, + {Key: testKey2, Value: mkVal("testValue2", ts[3])}, + {Key: testKey3, Value: mkVal("testValue3", ts[4])}, + {Key: testKey4, Value: mkVal("testValue4", ts[5])}, + {Key: testKey5, Value: mkVal("testValue5", ts[5])}, + {Key: testKey6, Value: mkVal("testValue5", ts[5])}, + } + for i, kv := range fixtureKVs { + v := *protoutil.Clone(&kv.Value).(*roachpb.Value) + v.Timestamp = hlc.Timestamp{} + if err := MVCCPut(ctx, engine, nil, kv.Key, kv.Value.Timestamp, hlc.ClockTimestamp{}, v, txnMap[i]); err != nil { + t.Fatal(err) + } + } - txn1ts := makeTxn(*txn1, ts[2]) - txn2ts := makeTxn(*txn2, ts[5]) - txnMap := map[int]*roachpb.Transaction{ - 2: txn1ts, - 5: txn2ts, - 6: txn2ts, - 7: txn2ts, - } + scanCases := []struct { + name string + consistent bool + txn *roachpb.Transaction + expIntents []roachpb.Intent + expValues []roachpb.KeyValue + }{ + { + name: "consistent-all-keys", + consistent: true, + txn: nil, + expIntents: []roachpb.Intent{ + roachpb.MakeIntent(&txn1ts.TxnMeta, testKey1), + roachpb.MakeIntent(&txn2ts.TxnMeta, testKey4), + }, + // would be []roachpb.KeyValue{fixtureKVs[3], fixtureKVs[4]} without WriteIntentError + expValues: nil, + }, + { + name: "consistent-txn1", + consistent: true, + txn: txn1ts, + expIntents: []roachpb.Intent{ + roachpb.MakeIntent(&txn2ts.TxnMeta, testKey4), + roachpb.MakeIntent(&txn2ts.TxnMeta, testKey5), + }, + expValues: nil, // []roachpb.KeyValue{fixtureKVs[2], fixtureKVs[3], fixtureKVs[4]}, + }, + { + name: "consistent-txn2", + consistent: true, + txn: txn2ts, + expIntents: []roachpb.Intent{ + roachpb.MakeIntent(&txn1ts.TxnMeta, testKey1), + }, + expValues: nil, // []roachpb.KeyValue{fixtureKVs[3], fixtureKVs[4], fixtureKVs[5]}, + }, + { + name: "inconsistent-all-keys", + consistent: false, + txn: nil, + expIntents: []roachpb.Intent{ + roachpb.MakeIntent(&txn1ts.TxnMeta, testKey1), + roachpb.MakeIntent(&txn2ts.TxnMeta, testKey4), + roachpb.MakeIntent(&txn2ts.TxnMeta, testKey5), + roachpb.MakeIntent(&txn2ts.TxnMeta, testKey6), + }, + expValues: []roachpb.KeyValue{fixtureKVs[0], fixtureKVs[3], fixtureKVs[4], fixtureKVs[1]}, + }, + } - fixtureKVs := []roachpb.KeyValue{ - {Key: testKey1, Value: mkVal("testValue1 pre", ts[0])}, - {Key: testKey4, Value: mkVal("testValue4 pre", ts[1])}, - {Key: testKey1, Value: mkVal("testValue1", ts[2])}, - {Key: testKey2, Value: mkVal("testValue2", ts[3])}, - {Key: testKey3, Value: mkVal("testValue3", ts[4])}, - {Key: testKey4, Value: mkVal("testValue4", ts[5])}, - {Key: testKey5, Value: mkVal("testValue5", ts[5])}, - {Key: testKey6, Value: mkVal("testValue5", ts[5])}, - } - for i, kv := range fixtureKVs { - v := *protoutil.Clone(&kv.Value).(*roachpb.Value) - v.Timestamp = hlc.Timestamp{} - if err := MVCCPut(ctx, engine, nil, kv.Key, kv.Value.Timestamp, hlc.ClockTimestamp{}, v, txnMap[i]); err != nil { - t.Fatal(err) - } + for _, scan := range scanCases { + t.Run(scan.name, func(t *testing.T) { + res, err := MVCCScan(ctx, engine, testKey1, testKey6.Next(), + hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Inconsistent: !scan.consistent, Txn: scan.txn, MaxIntents: 2}) + var wiErr *roachpb.WriteIntentError + _ = errors.As(err, &wiErr) + if (err == nil) != (wiErr == nil) { + t.Errorf("unexpected error: %+v", err) } - scanCases := []struct { - name string - consistent bool - txn *roachpb.Transaction - expIntents []roachpb.Intent - expValues []roachpb.KeyValue - }{ - { - name: "consistent-all-keys", - consistent: true, - txn: nil, - expIntents: []roachpb.Intent{ - roachpb.MakeIntent(&txn1ts.TxnMeta, testKey1), - roachpb.MakeIntent(&txn2ts.TxnMeta, testKey4), - }, - // would be []roachpb.KeyValue{fixtureKVs[3], fixtureKVs[4]} without WriteIntentError - expValues: nil, - }, - { - name: "consistent-txn1", - consistent: true, - txn: txn1ts, - expIntents: []roachpb.Intent{ - roachpb.MakeIntent(&txn2ts.TxnMeta, testKey4), - roachpb.MakeIntent(&txn2ts.TxnMeta, testKey5), - }, - expValues: nil, // []roachpb.KeyValue{fixtureKVs[2], fixtureKVs[3], fixtureKVs[4]}, - }, - { - name: "consistent-txn2", - consistent: true, - txn: txn2ts, - expIntents: []roachpb.Intent{ - roachpb.MakeIntent(&txn1ts.TxnMeta, testKey1), - }, - expValues: nil, // []roachpb.KeyValue{fixtureKVs[3], fixtureKVs[4], fixtureKVs[5]}, - }, - { - name: "inconsistent-all-keys", - consistent: false, - txn: nil, - expIntents: []roachpb.Intent{ - roachpb.MakeIntent(&txn1ts.TxnMeta, testKey1), - roachpb.MakeIntent(&txn2ts.TxnMeta, testKey4), - roachpb.MakeIntent(&txn2ts.TxnMeta, testKey5), - roachpb.MakeIntent(&txn2ts.TxnMeta, testKey6), - }, - expValues: []roachpb.KeyValue{fixtureKVs[0], fixtureKVs[3], fixtureKVs[4], fixtureKVs[1]}, - }, + if wiErr == nil != !scan.consistent { + t.Fatalf("expected write intent error; got %s", err) } - for _, scan := range scanCases { - t.Run(scan.name, func(t *testing.T) { - res, err := MVCCScan(ctx, engine, testKey1, testKey6.Next(), - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Inconsistent: !scan.consistent, Txn: scan.txn, MaxIntents: 2}) - var wiErr *roachpb.WriteIntentError - _ = errors.As(err, &wiErr) - if (err == nil) != (wiErr == nil) { - t.Errorf("unexpected error: %+v", err) - } - - if wiErr == nil != !scan.consistent { - t.Fatalf("expected write intent error; got %s", err) - } - - intents := res.Intents - kvs := res.KVs - if len(intents) > 0 != !scan.consistent { - t.Fatalf("expected different intents slice; got %+v", intents) - } + intents := res.Intents + kvs := res.KVs + if len(intents) > 0 != !scan.consistent { + t.Fatalf("expected different intents slice; got %+v", intents) + } - if scan.consistent { - intents = wiErr.Intents - } + if scan.consistent { + intents = wiErr.Intents + } - if !reflect.DeepEqual(intents, scan.expIntents) { - t.Fatalf("expected intents:\n%+v;\n got\n%+v", scan.expIntents, intents) - } + if !reflect.DeepEqual(intents, scan.expIntents) { + t.Fatalf("expected intents:\n%+v;\n got\n%+v", scan.expIntents, intents) + } - if !reflect.DeepEqual(kvs, scan.expValues) { - t.Fatalf("expected values %+v; got %+v", scan.expValues, kvs) - } - }) + if !reflect.DeepEqual(kvs, scan.expValues) { + t.Fatalf("expected values %+v; got %+v", scan.expValues, kvs) } }) } @@ -702,58 +649,54 @@ func TestMVCCGetInconsistent(t *testing.T) { ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - // Put two values to key 1, the latest with a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { - t.Fatal(err) - } + engine := NewDefaultInMemForTesting() + defer engine.Close() - // A get with consistent=false should fail in a txn. - if _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ - Inconsistent: true, - Txn: txn1, - }); err == nil { - t.Error("expected an error getting with consistent=false in txn") - } + // Put two values to key 1, the latest with a txn. + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + t.Fatal(err) + } - // Inconsistent get will fetch value1 for any timestamp. - for _, ts := range []hlc.Timestamp{{WallTime: 1}, {WallTime: 2}} { - val, intent, err := MVCCGet(ctx, engine, testKey1, ts, MVCCGetOptions{Inconsistent: true}) - if ts.Less(hlc.Timestamp{WallTime: 2}) { - if err != nil { - t.Fatal(err) - } - } else { - if intent == nil || !intent.Key.Equal(testKey1) { - t.Fatalf("expected %v, but got %v", testKey1, intent) - } - } - if !bytes.Equal(val.RawBytes, value1.RawBytes) { - t.Errorf("@%s expected %q; got %q", ts, value1.RawBytes, val.RawBytes) - } - } + // A get with consistent=false should fail in a txn. + if _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + Inconsistent: true, + Txn: txn1, + }); err == nil { + t.Error("expected an error getting with consistent=false in txn") + } - // Write a single intent for key 2 and verify get returns empty. - if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn2); err != nil { - t.Fatal(err) - } - val, intent, err := MVCCGet(ctx, engine, testKey2, hlc.Timestamp{WallTime: 2}, - MVCCGetOptions{Inconsistent: true}) - if intent == nil || !intent.Key.Equal(testKey2) { + // Inconsistent get will fetch value1 for any timestamp. + for _, ts := range []hlc.Timestamp{{WallTime: 1}, {WallTime: 2}} { + val, intent, err := MVCCGet(ctx, engine, testKey1, ts, MVCCGetOptions{Inconsistent: true}) + if ts.Less(hlc.Timestamp{WallTime: 2}) { + if err != nil { t.Fatal(err) } - if val != nil { - t.Errorf("expected empty val; got %+v", val) + } else { + if intent == nil || !intent.Key.Equal(testKey1) { + t.Fatalf("expected %v, but got %v", testKey1, intent) } - }) + } + if !bytes.Equal(val.RawBytes, value1.RawBytes) { + t.Errorf("@%s expected %q; got %q", ts, value1.RawBytes, val.RawBytes) + } + } + + // Write a single intent for key 2 and verify get returns empty. + if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn2); err != nil { + t.Fatal(err) + } + val, intent, err := MVCCGet(ctx, engine, testKey2, hlc.Timestamp{WallTime: 2}, + MVCCGetOptions{Inconsistent: true}) + if intent == nil || !intent.Key.Equal(testKey2) { + t.Fatal(err) + } + if val != nil { + t.Errorf("expected empty val; got %+v", val) } } @@ -764,109 +707,105 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - bytes1, err := protoutil.Marshal(&value1) - if err != nil { - t.Fatal(err) - } - bytes2, err := protoutil.Marshal(&value2) - if err != nil { - t.Fatal(err) - } + bytes1, err := protoutil.Marshal(&value1) + if err != nil { + t.Fatal(err) + } + bytes2, err := protoutil.Marshal(&value2) + if err != nil { + t.Fatal(err) + } - v1 := roachpb.MakeValueFromBytes(bytes1) - v2 := roachpb.MakeValueFromBytes(bytes2) + v1 := roachpb.MakeValueFromBytes(bytes1) + v2 := roachpb.MakeValueFromBytes(bytes2) - // Put two values to key 1, the latest with a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, v1, nil); err != nil { - t.Fatal(err) - } - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, v2, txn1ts); err != nil { - t.Fatal(err) - } + // Put two values to key 1, the latest with a txn. + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, v1, nil); err != nil { + t.Fatal(err) + } + txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, v2, txn1ts); err != nil { + t.Fatal(err) + } - // An inconsistent get should fail in a txn. - if _, err := MVCCGetProto(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, nil, MVCCGetOptions{ - Inconsistent: true, - Txn: txn1, - }); err == nil { - t.Error("expected an error getting inconsistently in txn") - } else if errors.HasType(err, (*roachpb.WriteIntentError)(nil)) { - t.Error("expected non-WriteIntentError with inconsistent read in txn") - } + // An inconsistent get should fail in a txn. + if _, err := MVCCGetProto(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, nil, MVCCGetOptions{ + Inconsistent: true, + Txn: txn1, + }); err == nil { + t.Error("expected an error getting inconsistently in txn") + } else if errors.HasType(err, (*roachpb.WriteIntentError)(nil)) { + t.Error("expected non-WriteIntentError with inconsistent read in txn") + } - // Inconsistent get will fetch value1 for any timestamp. + // Inconsistent get will fetch value1 for any timestamp. - for _, ts := range []hlc.Timestamp{{WallTime: 1}, {WallTime: 2}} { - val := roachpb.Value{} - found, err := MVCCGetProto(ctx, engine, testKey1, ts, &val, MVCCGetOptions{ - Inconsistent: true, - }) - if ts.Less(hlc.Timestamp{WallTime: 2}) { - if err != nil { - t.Fatal(err) - } - } else if err != nil { - t.Fatal(err) - } - if !found { - t.Errorf("expected to find result with inconsistent read") - } - valBytes, err := val.GetBytes() - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(valBytes, []byte("testValue1")) { - t.Errorf("@%s expected %q; got %q", ts, []byte("value1"), valBytes) - } + for _, ts := range []hlc.Timestamp{{WallTime: 1}, {WallTime: 2}} { + val := roachpb.Value{} + found, err := MVCCGetProto(ctx, engine, testKey1, ts, &val, MVCCGetOptions{ + Inconsistent: true, + }) + if ts.Less(hlc.Timestamp{WallTime: 2}) { + if err != nil { + t.Fatal(err) } + } else if err != nil { + t.Fatal(err) + } + if !found { + t.Errorf("expected to find result with inconsistent read") + } + valBytes, err := val.GetBytes() + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(valBytes, []byte("testValue1")) { + t.Errorf("@%s expected %q; got %q", ts, []byte("value1"), valBytes) + } + } - { - // Write a single intent for key 2 and verify get returns empty. - if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, hlc.ClockTimestamp{}, v1, txn2); err != nil { - t.Fatal(err) - } - val := roachpb.Value{} - found, err := MVCCGetProto(ctx, engine, testKey2, hlc.Timestamp{WallTime: 2}, &val, MVCCGetOptions{ - Inconsistent: true, - }) - if err != nil { - t.Fatal(err) - } - if found { - t.Errorf("expected no result; got %+v", val) - } - } + { + // Write a single intent for key 2 and verify get returns empty. + if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, hlc.ClockTimestamp{}, v1, txn2); err != nil { + t.Fatal(err) + } + val := roachpb.Value{} + found, err := MVCCGetProto(ctx, engine, testKey2, hlc.Timestamp{WallTime: 2}, &val, MVCCGetOptions{ + Inconsistent: true, + }) + if err != nil { + t.Fatal(err) + } + if found { + t.Errorf("expected no result; got %+v", val) + } + } - { - // Write a malformed value (not an encoded MVCCKeyValue) and a - // write intent to key 3; the parse error is returned instead of the - // write intent. - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey3, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, v2, txn1ts); err != nil { - t.Fatal(err) - } - val := roachpb.Value{} - found, err := MVCCGetProto(ctx, engine, testKey3, hlc.Timestamp{WallTime: 1}, &val, MVCCGetOptions{ - Inconsistent: true, - }) - if err == nil { - t.Errorf("expected error reading malformed data") - } else if !strings.HasPrefix(err.Error(), "proto: ") { - t.Errorf("expected proto error, got %s", err) - } - if !found { - t.Errorf("expected to find result with malformed data") - } - } + { + // Write a malformed value (not an encoded MVCCKeyValue) and a + // write intent to key 3; the parse error is returned instead of the + // write intent. + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey3, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, v2, txn1ts); err != nil { + t.Fatal(err) + } + val := roachpb.Value{} + found, err := MVCCGetProto(ctx, engine, testKey3, hlc.Timestamp{WallTime: 1}, &val, MVCCGetOptions{ + Inconsistent: true, }) + if err == nil { + t.Errorf("expected error reading malformed data") + } else if !strings.HasPrefix(err.Error(), "proto: ") { + t.Errorf("expected proto error, got %s", err) + } + if !found { + t.Errorf("expected to find result with malformed data") + } } } @@ -878,67 +817,63 @@ func TestMVCCInvalidateIterator(t *testing.T) { for _, which := range []string{"get", "scan", "findSplitKey", "computeStats"} { t.Run(which, func(t *testing.T) { - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - ctx := context.Background() - ts1 := hlc.Timestamp{WallTime: 1} - ts2 := hlc.Timestamp{WallTime: 2} + engine := NewDefaultInMemForTesting() + defer engine.Close() - key := roachpb.Key("a") - if err := MVCCPut(ctx, engine, nil, key, ts1, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } + ctx := context.Background() + ts1 := hlc.Timestamp{WallTime: 1} + ts2 := hlc.Timestamp{WallTime: 2} - var iterOptions IterOptions - switch which { - case "get": - iterOptions.Prefix = true - case "computeStats": - iterOptions.KeyTypes = IterKeyTypePointsAndRanges - iterOptions.UpperBound = roachpb.KeyMax - case "scan", "findSplitKey": - iterOptions.UpperBound = roachpb.KeyMax - } + key := roachpb.Key("a") + if err := MVCCPut(ctx, engine, nil, key, ts1, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } - // Use a batch which internally caches the iterator. - batch := engine.NewBatch() - defer batch.Close() + var iterOptions IterOptions + switch which { + case "get": + iterOptions.Prefix = true + case "computeStats": + iterOptions.KeyTypes = IterKeyTypePointsAndRanges + iterOptions.UpperBound = roachpb.KeyMax + case "scan", "findSplitKey": + iterOptions.UpperBound = roachpb.KeyMax + } - { - // Seek the iter to a valid position. - iter := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, iterOptions) - iter.SeekGE(MakeMVCCMetadataKey(key)) - iter.Close() - } + // Use a batch which internally caches the iterator. + batch := engine.NewBatch() + defer batch.Close() - var err error - switch which { - case "get": - _, _, err = MVCCGet(ctx, batch, key, ts2, MVCCGetOptions{}) - case "scan": - _, err = MVCCScan(ctx, batch, key, roachpb.KeyMax, ts2, MVCCScanOptions{}) - case "findSplitKey": - _, err = MVCCFindSplitKey(ctx, batch, roachpb.RKeyMin, roachpb.RKeyMax, 64<<20) - case "computeStatsForIter": - iter := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, iterOptions) - iter.SeekGE(MVCCKey{Key: iterOptions.LowerBound}) - _, err = ComputeStatsForIter(iter, 0) - iter.Close() - } - if err != nil { - t.Fatal(err) - } + { + // Seek the iter to a valid position. + iter := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, iterOptions) + iter.SeekGE(MakeMVCCMetadataKey(key)) + iter.Close() + } + + var err error + switch which { + case "get": + _, _, err = MVCCGet(ctx, batch, key, ts2, MVCCGetOptions{}) + case "scan": + _, err = MVCCScan(ctx, batch, key, roachpb.KeyMax, ts2, MVCCScanOptions{}) + case "findSplitKey": + _, err = MVCCFindSplitKey(ctx, batch, roachpb.RKeyMin, roachpb.RKeyMax, 64<<20) + case "computeStatsForIter": + iter := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, iterOptions) + iter.SeekGE(MVCCKey{Key: iterOptions.LowerBound}) + _, err = ComputeStatsForIter(iter, 0) + iter.Close() + } + if err != nil { + t.Fatal(err) + } - // Verify that the iter is invalid. - iter := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, iterOptions) - defer iter.Close() - if ok, _ := iter.Valid(); ok { - t.Fatalf("iterator should not be valid") - } - }) + // Verify that the iter is invalid. + iter := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, iterOptions) + defer iter.Close() + if ok, _ := iter.Valid(); ok { + t.Fatalf("iterator should not be valid") } }) } @@ -1046,14 +981,9 @@ func TestMVCCScan(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - mvccScanTest(ctx, t, engine) - }) - } + engine := NewDefaultInMemForTesting() + defer engine.Close() + mvccScanTest(ctx, t, engine) } func TestMVCCScanMaxNum(t *testing.T) { @@ -1061,79 +991,75 @@ func TestMVCCScanMaxNum(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + t.Fatal(err) + } - res, err := MVCCScan(ctx, engine, testKey2, testKey4, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: 1}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 || - !bytes.Equal(res.KVs[0].Key, testKey2) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) { - t.Fatal("the value should not be empty") - } - if expected := (roachpb.Span{Key: testKey3, EndKey: testKey4}); !res.ResumeSpan.EqualValue(expected) { - t.Fatalf("expected = %+v, resumeSpan = %+v", expected, res.ResumeSpan) - } + res, err := MVCCScan(ctx, engine, testKey2, testKey4, + hlc.Timestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: 1}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 || + !bytes.Equal(res.KVs[0].Key, testKey2) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) { + t.Fatal("the value should not be empty") + } + if expected := (roachpb.Span{Key: testKey3, EndKey: testKey4}); !res.ResumeSpan.EqualValue(expected) { + t.Fatalf("expected = %+v, resumeSpan = %+v", expected, res.ResumeSpan) + } - res, err = MVCCScan(ctx, engine, testKey2, testKey4, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: -1}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 0 { - t.Fatal("the value should be empty") - } - if expected := (roachpb.Span{Key: testKey2, EndKey: testKey4}); !res.ResumeSpan.EqualValue(expected) { - t.Fatalf("expected = %+v, resumeSpan = %+v", expected, res.ResumeSpan) - } + res, err = MVCCScan(ctx, engine, testKey2, testKey4, + hlc.Timestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: -1}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 0 { + t.Fatal("the value should be empty") + } + if expected := (roachpb.Span{Key: testKey2, EndKey: testKey4}); !res.ResumeSpan.EqualValue(expected) { + t.Fatalf("expected = %+v, resumeSpan = %+v", expected, res.ResumeSpan) + } - // Note: testKey6, though not scanned directly, is important in testing that - // the computed resume span does not extend beyond the upper bound of a scan. - res, err = MVCCScan(ctx, engine, testKey4, testKey5, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: 1}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 { - t.Fatalf("expected 1 key but got %d", len(res.KVs)) - } - if res.ResumeSpan != nil { - t.Fatalf("resumeSpan = %+v", res.ResumeSpan) - } + // Note: testKey6, though not scanned directly, is important in testing that + // the computed resume span does not extend beyond the upper bound of a scan. + res, err = MVCCScan(ctx, engine, testKey4, testKey5, + hlc.Timestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: 1}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 { + t.Fatalf("expected 1 key but got %d", len(res.KVs)) + } + if res.ResumeSpan != nil { + t.Fatalf("resumeSpan = %+v", res.ResumeSpan) + } - res, err = MVCCScan(ctx, engine, testKey5, testKey6.Next(), - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true, MaxKeys: 1}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 { - t.Fatalf("expected 1 key but got %d", len(res.KVs)) - } - if res.ResumeSpan != nil { - t.Fatalf("resumeSpan = %+v", res.ResumeSpan) - } - }) + res, err = MVCCScan(ctx, engine, testKey5, testKey6.Next(), + hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true, MaxKeys: 1}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 { + t.Fatalf("expected 1 key but got %d", len(res.KVs)) + } + if res.ResumeSpan != nil { + t.Fatalf("resumeSpan = %+v", res.ResumeSpan) } } @@ -1142,51 +1068,47 @@ func TestMVCCScanWithKeyPrefix(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Let's say you have: - // a - // a - // a - // aa - // aa - // aa - // b - // b - // In this case, if we scan from "a"-"b", we wish to skip - // a and a and find "aa'. - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/b"), hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { - t.Fatal(err) - } + // Let's say you have: + // a + // a + // a + // aa + // aa + // aa + // b + // b + // In this case, if we scan from "a"-"b", we wish to skip + // a and a and find "aa'. + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/b"), hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + t.Fatal(err) + } - res, err := MVCCScan(ctx, engine, roachpb.Key("/a"), roachpb.Key("/b"), - hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 2 || - !bytes.Equal(res.KVs[0].Key, roachpb.Key("/a")) || - !bytes.Equal(res.KVs[1].Key, roachpb.Key("/aa")) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) || - !bytes.Equal(res.KVs[1].Value.RawBytes, value2.RawBytes) { - t.Fatal("the value should not be empty") - } - }) + res, err := MVCCScan(ctx, engine, roachpb.Key("/a"), roachpb.Key("/b"), + hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 2 || + !bytes.Equal(res.KVs[0].Key, roachpb.Key("/a")) || + !bytes.Equal(res.KVs[1].Key, roachpb.Key("/aa")) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) || + !bytes.Equal(res.KVs[1].Value.RawBytes, value2.RawBytes) { + t.Fatal("the value should not be empty") } } @@ -1195,44 +1117,40 @@ func TestMVCCScanInTxn(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + t.Fatal(err) + } - res, err := MVCCScan(ctx, engine, testKey2, testKey4, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Txn: txn1}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 2 || - !bytes.Equal(res.KVs[0].Key, testKey2) || - !bytes.Equal(res.KVs[1].Key, testKey3) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) || - !bytes.Equal(res.KVs[1].Value.RawBytes, value3.RawBytes) { - t.Fatal("the value should not be empty") - } + res, err := MVCCScan(ctx, engine, testKey2, testKey4, + hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Txn: txn1}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 2 || + !bytes.Equal(res.KVs[0].Key, testKey2) || + !bytes.Equal(res.KVs[1].Key, testKey3) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) || + !bytes.Equal(res.KVs[1].Value.RawBytes, value3.RawBytes) { + t.Fatal("the value should not be empty") + } - if _, err := MVCCScan( - ctx, engine, testKey2, testKey4, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}, - ); err == nil { - t.Fatal("expected error on uncommitted write intent") - } - }) + if _, err := MVCCScan( + ctx, engine, testKey2, testKey4, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}, + ); err == nil { + t.Fatal("expected error on uncommitted write intent") } } @@ -1243,90 +1161,86 @@ func TestMVCCScanInconsistent(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // A scan with consistent=false should fail in a txn. - if _, err := MVCCScan( - ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 1}, - MVCCScanOptions{Inconsistent: true, Txn: txn1}, - ); err == nil { - t.Error("expected an error scanning with consistent=false in txn") - } + // A scan with consistent=false should fail in a txn. + if _, err := MVCCScan( + ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 1}, + MVCCScanOptions{Inconsistent: true, Txn: txn1}, + ); err == nil { + t.Error("expected an error scanning with consistent=false in txn") + } + + ts1 := hlc.Timestamp{WallTime: 1} + ts2 := hlc.Timestamp{WallTime: 2} + ts3 := hlc.Timestamp{WallTime: 3} + ts4 := hlc.Timestamp{WallTime: 4} + ts5 := hlc.Timestamp{WallTime: 5} + ts6 := hlc.Timestamp{WallTime: 6} + if err := MVCCPut(ctx, engine, nil, testKey1, ts1, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + txn1ts2 := makeTxn(*txn1, ts2) + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts2); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, ts3, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, ts4, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + txn2ts5 := makeTxn(*txn2, ts5) + if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts5.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2ts5); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey4, ts6, hlc.ClockTimestamp{}, value4, nil); err != nil { + t.Fatal(err) + } - ts1 := hlc.Timestamp{WallTime: 1} - ts2 := hlc.Timestamp{WallTime: 2} - ts3 := hlc.Timestamp{WallTime: 3} - ts4 := hlc.Timestamp{WallTime: 4} - ts5 := hlc.Timestamp{WallTime: 5} - ts6 := hlc.Timestamp{WallTime: 6} - if err := MVCCPut(ctx, engine, nil, testKey1, ts1, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - txn1ts2 := makeTxn(*txn1, ts2) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts2); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, ts3, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, ts4, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - txn2ts5 := makeTxn(*txn2, ts5) - if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts5.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2ts5); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey4, ts6, hlc.ClockTimestamp{}, value4, nil); err != nil { - t.Fatal(err) - } - - expIntents := []roachpb.Intent{ - roachpb.MakeIntent(&txn1ts2.TxnMeta, testKey1), - roachpb.MakeIntent(&txn2ts5.TxnMeta, testKey3), - } - res, err := MVCCScan( - ctx, engine, testKey1, testKey4.Next(), hlc.Timestamp{WallTime: 7}, - MVCCScanOptions{Inconsistent: true}, - ) - if err != nil { - t.Fatal(err) - } - if !reflect.DeepEqual(res.Intents, expIntents) { - t.Fatalf("expected %v, but found %v", expIntents, res.Intents) - } + expIntents := []roachpb.Intent{ + roachpb.MakeIntent(&txn1ts2.TxnMeta, testKey1), + roachpb.MakeIntent(&txn2ts5.TxnMeta, testKey3), + } + res, err := MVCCScan( + ctx, engine, testKey1, testKey4.Next(), hlc.Timestamp{WallTime: 7}, + MVCCScanOptions{Inconsistent: true}, + ) + if err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(res.Intents, expIntents) { + t.Fatalf("expected %v, but found %v", expIntents, res.Intents) + } - makeTimestampedValue := func(v roachpb.Value, ts hlc.Timestamp) roachpb.Value { - v.Timestamp = ts - return v - } + makeTimestampedValue := func(v roachpb.Value, ts hlc.Timestamp) roachpb.Value { + v.Timestamp = ts + return v + } - expKVs := []roachpb.KeyValue{ - {Key: testKey1, Value: makeTimestampedValue(value1, ts1)}, - {Key: testKey2, Value: makeTimestampedValue(value2, ts4)}, - {Key: testKey4, Value: makeTimestampedValue(value4, ts6)}, - } - if !reflect.DeepEqual(res.KVs, expKVs) { - t.Errorf("expected key values equal %v != %v", res.KVs, expKVs) - } + expKVs := []roachpb.KeyValue{ + {Key: testKey1, Value: makeTimestampedValue(value1, ts1)}, + {Key: testKey2, Value: makeTimestampedValue(value2, ts4)}, + {Key: testKey4, Value: makeTimestampedValue(value4, ts6)}, + } + if !reflect.DeepEqual(res.KVs, expKVs) { + t.Errorf("expected key values equal %v != %v", res.KVs, expKVs) + } - // Now try a scan at a historical timestamp. - expIntents = expIntents[:1] - res, err = MVCCScan(ctx, engine, testKey1, testKey4.Next(), - hlc.Timestamp{WallTime: 3}, MVCCScanOptions{Inconsistent: true}) - if !reflect.DeepEqual(res.Intents, expIntents) { - t.Fatal(err) - } - expKVs = []roachpb.KeyValue{ - {Key: testKey1, Value: makeTimestampedValue(value1, ts1)}, - {Key: testKey2, Value: makeTimestampedValue(value1, ts3)}, - } - if !reflect.DeepEqual(res.KVs, expKVs) { - t.Errorf("expected key values equal %v != %v", res.Intents, expKVs) - } - }) + // Now try a scan at a historical timestamp. + expIntents = expIntents[:1] + res, err = MVCCScan(ctx, engine, testKey1, testKey4.Next(), + hlc.Timestamp{WallTime: 3}, MVCCScanOptions{Inconsistent: true}) + if !reflect.DeepEqual(res.Intents, expIntents) { + t.Fatal(err) + } + expKVs = []roachpb.KeyValue{ + {Key: testKey1, Value: makeTimestampedValue(value1, ts1)}, + {Key: testKey2, Value: makeTimestampedValue(value1, ts3)}, + } + if !reflect.DeepEqual(res.KVs, expKVs) { + t.Errorf("expected key values equal %v != %v", res.Intents, expKVs) } } @@ -1335,163 +1249,159 @@ func TestMVCCDeleteRange(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value5, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value5, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { + t.Fatal(err) + } - // Attempt to delete two keys. - deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - 2, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) - if err != nil { - t.Fatal(err) - } - if deleted != nil { - t.Fatal("the value should be empty") - } - if num != 2 { - t.Fatalf("incorrect number of keys deleted: %d", num) - } - if expected := (roachpb.Span{Key: testKey4, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { - t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) - } - res, _ := MVCCScan(ctx, engine, localMax, keyMax, - hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) - if len(res.KVs) != 4 || - !bytes.Equal(res.KVs[0].Key, testKey1) || - !bytes.Equal(res.KVs[1].Key, testKey4) || - !bytes.Equal(res.KVs[2].Key, testKey5) || - !bytes.Equal(res.KVs[3].Key, testKey6) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) || - !bytes.Equal(res.KVs[1].Value.RawBytes, value4.RawBytes) || - !bytes.Equal(res.KVs[2].Value.RawBytes, value5.RawBytes) || - !bytes.Equal(res.KVs[3].Value.RawBytes, value6.RawBytes) { - t.Fatal("the value should not be empty") - } + // Attempt to delete two keys. + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) + if err != nil { + t.Fatal(err) + } + if deleted != nil { + t.Fatal("the value should be empty") + } + if num != 2 { + t.Fatalf("incorrect number of keys deleted: %d", num) + } + if expected := (roachpb.Span{Key: testKey4, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { + t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) + } + res, _ := MVCCScan(ctx, engine, localMax, keyMax, + hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) + if len(res.KVs) != 4 || + !bytes.Equal(res.KVs[0].Key, testKey1) || + !bytes.Equal(res.KVs[1].Key, testKey4) || + !bytes.Equal(res.KVs[2].Key, testKey5) || + !bytes.Equal(res.KVs[3].Key, testKey6) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) || + !bytes.Equal(res.KVs[1].Value.RawBytes, value4.RawBytes) || + !bytes.Equal(res.KVs[2].Value.RawBytes, value5.RawBytes) || + !bytes.Equal(res.KVs[3].Value.RawBytes, value6.RawBytes) { + t.Fatal("the value should not be empty") + } - // Try again, but with tombstones set to true to fetch the deleted keys as well. - kvs := []roachpb.KeyValue{} - if _, err = MVCCIterate( - ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{Tombstones: true}, - func(kv roachpb.KeyValue) error { - kvs = append(kvs, kv) - return nil - }, - ); err != nil { - t.Fatal(err) - } - if len(kvs) != 6 || - !bytes.Equal(kvs[0].Key, testKey1) || - !bytes.Equal(kvs[1].Key, testKey2) || - !bytes.Equal(kvs[2].Key, testKey3) || - !bytes.Equal(kvs[3].Key, testKey4) || - !bytes.Equal(kvs[4].Key, testKey5) || - !bytes.Equal(kvs[5].Key, testKey6) || - !bytes.Equal(kvs[0].Value.RawBytes, value1.RawBytes) || - !bytes.Equal(kvs[1].Value.RawBytes, nil) || - !bytes.Equal(kvs[2].Value.RawBytes, nil) || - !bytes.Equal(kvs[3].Value.RawBytes, value4.RawBytes) || - !bytes.Equal(kvs[4].Value.RawBytes, value5.RawBytes) || - !bytes.Equal(kvs[5].Value.RawBytes, value6.RawBytes) { - t.Fatal("the value should not be empty") - } + // Try again, but with tombstones set to true to fetch the deleted keys as well. + kvs := []roachpb.KeyValue{} + if _, err = MVCCIterate( + ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{Tombstones: true}, + func(kv roachpb.KeyValue) error { + kvs = append(kvs, kv) + return nil + }, + ); err != nil { + t.Fatal(err) + } + if len(kvs) != 6 || + !bytes.Equal(kvs[0].Key, testKey1) || + !bytes.Equal(kvs[1].Key, testKey2) || + !bytes.Equal(kvs[2].Key, testKey3) || + !bytes.Equal(kvs[3].Key, testKey4) || + !bytes.Equal(kvs[4].Key, testKey5) || + !bytes.Equal(kvs[5].Key, testKey6) || + !bytes.Equal(kvs[0].Value.RawBytes, value1.RawBytes) || + !bytes.Equal(kvs[1].Value.RawBytes, nil) || + !bytes.Equal(kvs[2].Value.RawBytes, nil) || + !bytes.Equal(kvs[3].Value.RawBytes, value4.RawBytes) || + !bytes.Equal(kvs[4].Value.RawBytes, value5.RawBytes) || + !bytes.Equal(kvs[5].Value.RawBytes, value6.RawBytes) { + t.Fatal("the value should not be empty") + } - // Attempt to delete no keys. - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - -1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) - if err != nil { - t.Fatal(err) - } - if deleted != nil { - t.Fatal("the value should be empty") - } - if num != 0 { - t.Fatalf("incorrect number of keys deleted: %d", num) - } - if expected := (roachpb.Span{Key: testKey2, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { - t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) - } - res, _ = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, - MVCCScanOptions{}) - if len(res.KVs) != 4 || - !bytes.Equal(res.KVs[0].Key, testKey1) || - !bytes.Equal(res.KVs[1].Key, testKey4) || - !bytes.Equal(res.KVs[2].Key, testKey5) || - !bytes.Equal(res.KVs[3].Key, testKey6) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) || - !bytes.Equal(res.KVs[1].Value.RawBytes, value4.RawBytes) || - !bytes.Equal(res.KVs[2].Value.RawBytes, value5.RawBytes) || - !bytes.Equal(res.KVs[3].Value.RawBytes, value6.RawBytes) { - t.Fatal("the value should not be empty") - } + // Attempt to delete no keys. + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + -1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) + if err != nil { + t.Fatal(err) + } + if deleted != nil { + t.Fatal("the value should be empty") + } + if num != 0 { + t.Fatalf("incorrect number of keys deleted: %d", num) + } + if expected := (roachpb.Span{Key: testKey2, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { + t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) + } + res, _ = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, + MVCCScanOptions{}) + if len(res.KVs) != 4 || + !bytes.Equal(res.KVs[0].Key, testKey1) || + !bytes.Equal(res.KVs[1].Key, testKey4) || + !bytes.Equal(res.KVs[2].Key, testKey5) || + !bytes.Equal(res.KVs[3].Key, testKey6) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) || + !bytes.Equal(res.KVs[1].Value.RawBytes, value4.RawBytes) || + !bytes.Equal(res.KVs[2].Value.RawBytes, value5.RawBytes) || + !bytes.Equal(res.KVs[3].Value.RawBytes, value6.RawBytes) { + t.Fatal("the value should not be empty") + } - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey4, keyMax, - 0, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) - if err != nil { - t.Fatal(err) - } - if deleted != nil { - t.Fatal("the value should be empty") - } - if num != 3 { - t.Fatalf("incorrect number of keys deleted: %d", num) - } - if resumeSpan != nil { - t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) - } - res, err = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, - MVCCScanOptions{}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 || - !bytes.Equal(res.KVs[0].Key, testKey1) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) { - t.Fatalf("the value should not be empty: %+v", res.KVs) - } + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey4, keyMax, + 0, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) + if err != nil { + t.Fatal(err) + } + if deleted != nil { + t.Fatal("the value should be empty") + } + if num != 3 { + t.Fatalf("incorrect number of keys deleted: %d", num) + } + if resumeSpan != nil { + t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) + } + res, err = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, + MVCCScanOptions{}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 || + !bytes.Equal(res.KVs[0].Key, testKey1) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) { + t.Fatalf("the value should not be empty: %+v", res.KVs) + } - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, localMax, testKey2, - 0, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) - if err != nil { - t.Fatal(err) - } - if deleted != nil { - t.Fatal("the value should not be empty") - } - if num != 1 { - t.Fatalf("incorrect number of keys deleted: %d", num) - } - if resumeSpan != nil { - t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) - } - res, _ = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, - MVCCScanOptions{}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 0 { - t.Fatal("the value should be empty") - } - }) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, localMax, testKey2, + 0, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, false) + if err != nil { + t.Fatal(err) + } + if deleted != nil { + t.Fatal("the value should not be empty") + } + if num != 1 { + t.Fatalf("incorrect number of keys deleted: %d", num) + } + if resumeSpan != nil { + t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) + } + res, _ = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, + MVCCScanOptions{}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 0 { + t.Fatal("the value should be empty") } } @@ -1500,148 +1410,144 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value5, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value5, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { + t.Fatal(err) + } - // Attempt to delete two keys. - deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - 2, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) - if err != nil { - t.Fatal(err) - } - if len(deleted) != 2 { - t.Fatal("the value should not be empty") - } - if num != 2 { - t.Fatalf("incorrect number of keys deleted: %d", num) - } - if expected, actual := testKey2, deleted[0]; !expected.Equal(actual) { - t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) - } - if expected, actual := testKey3, deleted[1]; !expected.Equal(actual) { - t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) - } - if expected := (roachpb.Span{Key: testKey4, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { - t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) - } - res, _ := MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, - MVCCScanOptions{}) - if len(res.KVs) != 4 || - !bytes.Equal(res.KVs[0].Key, testKey1) || - !bytes.Equal(res.KVs[1].Key, testKey4) || - !bytes.Equal(res.KVs[2].Key, testKey5) || - !bytes.Equal(res.KVs[3].Key, testKey6) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) || - !bytes.Equal(res.KVs[1].Value.RawBytes, value4.RawBytes) || - !bytes.Equal(res.KVs[2].Value.RawBytes, value5.RawBytes) || - !bytes.Equal(res.KVs[3].Value.RawBytes, value6.RawBytes) { - t.Fatal("the value should not be empty") - } + // Attempt to delete two keys. + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) + if err != nil { + t.Fatal(err) + } + if len(deleted) != 2 { + t.Fatal("the value should not be empty") + } + if num != 2 { + t.Fatalf("incorrect number of keys deleted: %d", num) + } + if expected, actual := testKey2, deleted[0]; !expected.Equal(actual) { + t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) + } + if expected, actual := testKey3, deleted[1]; !expected.Equal(actual) { + t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) + } + if expected := (roachpb.Span{Key: testKey4, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { + t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) + } + res, _ := MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, + MVCCScanOptions{}) + if len(res.KVs) != 4 || + !bytes.Equal(res.KVs[0].Key, testKey1) || + !bytes.Equal(res.KVs[1].Key, testKey4) || + !bytes.Equal(res.KVs[2].Key, testKey5) || + !bytes.Equal(res.KVs[3].Key, testKey6) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) || + !bytes.Equal(res.KVs[1].Value.RawBytes, value4.RawBytes) || + !bytes.Equal(res.KVs[2].Value.RawBytes, value5.RawBytes) || + !bytes.Equal(res.KVs[3].Value.RawBytes, value6.RawBytes) { + t.Fatal("the value should not be empty") + } - // Attempt to delete no keys. - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - -1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) - if err != nil { - t.Fatal(err) - } - if deleted != nil { - t.Fatalf("the value should be empty: %s", deleted) - } - if num != 0 { - t.Fatalf("incorrect number of keys deleted: %d", num) - } - if expected := (roachpb.Span{Key: testKey2, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { - t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) - } - res, _ = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, - MVCCScanOptions{}) - if len(res.KVs) != 4 || - !bytes.Equal(res.KVs[0].Key, testKey1) || - !bytes.Equal(res.KVs[1].Key, testKey4) || - !bytes.Equal(res.KVs[2].Key, testKey5) || - !bytes.Equal(res.KVs[3].Key, testKey6) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) || - !bytes.Equal(res.KVs[1].Value.RawBytes, value4.RawBytes) || - !bytes.Equal(res.KVs[2].Value.RawBytes, value5.RawBytes) || - !bytes.Equal(res.KVs[3].Value.RawBytes, value6.RawBytes) { - t.Fatal("the value should not be empty") - } + // Attempt to delete no keys. + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + -1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) + if err != nil { + t.Fatal(err) + } + if deleted != nil { + t.Fatalf("the value should be empty: %s", deleted) + } + if num != 0 { + t.Fatalf("incorrect number of keys deleted: %d", num) + } + if expected := (roachpb.Span{Key: testKey2, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { + t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) + } + res, _ = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, + MVCCScanOptions{}) + if len(res.KVs) != 4 || + !bytes.Equal(res.KVs[0].Key, testKey1) || + !bytes.Equal(res.KVs[1].Key, testKey4) || + !bytes.Equal(res.KVs[2].Key, testKey5) || + !bytes.Equal(res.KVs[3].Key, testKey6) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) || + !bytes.Equal(res.KVs[1].Value.RawBytes, value4.RawBytes) || + !bytes.Equal(res.KVs[2].Value.RawBytes, value5.RawBytes) || + !bytes.Equal(res.KVs[3].Value.RawBytes, value6.RawBytes) { + t.Fatal("the value should not be empty") + } - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey4, keyMax, - math.MaxInt64, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) - if err != nil { - t.Fatal(err) - } - if len(deleted) != 3 { - t.Fatal("the value should not be empty") - } - if num != 3 { - t.Fatalf("incorrect number of keys deleted: %d", num) - } - if expected, actual := testKey4, deleted[0]; !expected.Equal(actual) { - t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) - } - if expected, actual := testKey5, deleted[1]; !expected.Equal(actual) { - t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) - } - if expected, actual := testKey6, deleted[2]; !expected.Equal(actual) { - t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) - } - if resumeSpan != nil { - t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) - } - res, _ = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, - MVCCScanOptions{}) - if len(res.KVs) != 1 || - !bytes.Equal(res.KVs[0].Key, testKey1) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) { - t.Fatal("the value should not be empty") - } + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, testKey4, keyMax, + math.MaxInt64, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) + if err != nil { + t.Fatal(err) + } + if len(deleted) != 3 { + t.Fatal("the value should not be empty") + } + if num != 3 { + t.Fatalf("incorrect number of keys deleted: %d", num) + } + if expected, actual := testKey4, deleted[0]; !expected.Equal(actual) { + t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) + } + if expected, actual := testKey5, deleted[1]; !expected.Equal(actual) { + t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) + } + if expected, actual := testKey6, deleted[2]; !expected.Equal(actual) { + t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) + } + if resumeSpan != nil { + t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) + } + res, _ = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, + MVCCScanOptions{}) + if len(res.KVs) != 1 || + !bytes.Equal(res.KVs[0].Key, testKey1) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) { + t.Fatal("the value should not be empty") + } - deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, localMax, testKey2, - math.MaxInt64, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) - if err != nil { - t.Fatal(err) - } - if len(deleted) != 1 { - t.Fatal("the value should not be empty") - } - if num != 1 { - t.Fatalf("incorrect number of keys deleted: %d", num) - } - if expected, actual := testKey1, deleted[0]; !expected.Equal(actual) { - t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) - } - if resumeSpan != nil { - t.Fatalf("wrong resume key: %v", resumeSpan) - } - res, _ = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, - MVCCScanOptions{}) - if len(res.KVs) != 0 { - t.Fatal("the value should be empty") - } - }) + deleted, resumeSpan, num, err = MVCCDeleteRange(ctx, engine, nil, localMax, testKey2, + math.MaxInt64, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true) + if err != nil { + t.Fatal(err) + } + if len(deleted) != 1 { + t.Fatal("the value should not be empty") + } + if num != 1 { + t.Fatalf("incorrect number of keys deleted: %d", num) + } + if expected, actual := testKey1, deleted[0]; !expected.Equal(actual) { + t.Fatalf("wrong key deleted: expected %v found %v", expected, actual) + } + if resumeSpan != nil { + t.Fatalf("wrong resume key: %v", resumeSpan) + } + res, _ = MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, + MVCCScanOptions{}) + if len(res.KVs) != 0 { + t.Fatal("the value should be empty") } } @@ -1650,38 +1556,34 @@ func TestMVCCDeleteRangeFailed(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey2, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn); err != nil { - t.Fatal(err) - } - txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { - t.Fatal(err) - } + txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + txn.Sequence++ + if err := MVCCPut(ctx, engine, nil, testKey2, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn); err != nil { + t.Fatal(err) + } + txn.Sequence++ + if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + t.Fatal(err) + } - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, - math.MaxInt64, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil, false); err == nil { - t.Fatal("expected error on uncommitted write intent") - } + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, nil, false); err == nil { + t.Fatal("expected error on uncommitted write intent") + } - txn.Sequence++ - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, - math.MaxInt64, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, false); err != nil { - t.Fatal(err) - } - }) + txn.Sequence++ + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, + math.MaxInt64, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, false); err != nil { + t.Fatal(err) } } @@ -1690,33 +1592,29 @@ func TestMVCCDeleteRangeConcurrentTxn(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - txn2ts := makeTxn(*txn2, hlc.Timestamp{WallTime: 2}) + txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn2ts := makeTxn(*txn2, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2ts); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2ts); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value4, nil); err != nil { + t.Fatal(err) + } - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, - math.MaxInt64, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, txn1ts, false, - ); err == nil { - t.Fatal("expected error on uncommitted write intent") - } - }) + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey4, + math.MaxInt64, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, txn1ts, false, + ); err == nil { + t.Fatal("expected error on uncommitted write intent") } } @@ -1727,39 +1625,35 @@ func TestMVCCUncommittedDeleteRangeVisible(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + t.Fatal(err) + } - if _, err := MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, hlc.ClockTimestamp{}, nil); err != nil { - t.Fatal(err) - } + if _, err := MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, hlc.ClockTimestamp{}, nil); err != nil { + t.Fatal(err) + } - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 2, Logical: 2}) - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey1, testKey4, - math.MaxInt64, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, false, - ); err != nil { - t.Fatal(err) - } + txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 2, Logical: 2}) + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey1, testKey4, + math.MaxInt64, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn, false, + ); err != nil { + t.Fatal(err) + } - txn.Epoch++ - res, _ := MVCCScan(ctx, engine, testKey1, testKey4, - hlc.Timestamp{WallTime: 3}, MVCCScanOptions{Txn: txn}) - if e := 2; len(res.KVs) != e { - t.Fatalf("e = %d, got %d", e, len(res.KVs)) - } - }) + txn.Epoch++ + res, _ := MVCCScan(ctx, engine, testKey1, testKey4, + hlc.Timestamp{WallTime: 3}, MVCCScanOptions{Txn: txn}) + if e := 2; len(res.KVs) != e { + t.Fatalf("e = %d, got %d", e, len(res.KVs)) } } @@ -1770,57 +1664,53 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) - if err != nil { - t.Fatal(err) - } - err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) - if err != nil { - t.Fatal(err) - } - _, err = MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil) - if err != nil { - t.Fatal(err) - } - - // Delete at a time before the tombstone. Should return a WriteTooOld error. - b := engine.NewBatch() - defer b.Close() - keys, resume, keyCount, err := MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, - math.MaxInt64, hlc.Timestamp{WallTime: 4}, hlc.ClockTimestamp{}, nil, true) - require.Nil(t, keys) - require.Nil(t, resume) - require.Equal(t, int64(0), keyCount) - require.NotNil(t, err) - require.IsType(t, (*roachpb.WriteTooOldError)(nil), err) - - // Delete at the same time as the tombstone. Should return a WriteTooOld error. - b = engine.NewBatch() - defer b.Close() - keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, - math.MaxInt64, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil, true) - require.Nil(t, keys) - require.Nil(t, resume) - require.Equal(t, int64(0), keyCount) - require.NotNil(t, err) - require.IsType(t, (*roachpb.WriteTooOldError)(nil), err) - - // Delete at a time after the tombstone. Should succeed and should not - // include the tombstone in the returned keys. - b = engine.NewBatch() - defer b.Close() - keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, - math.MaxInt64, hlc.Timestamp{WallTime: 6}, hlc.ClockTimestamp{}, nil, true) - require.Equal(t, []roachpb.Key{testKey1}, keys) - require.Nil(t, resume) - require.Equal(t, int64(1), keyCount) - require.NoError(t, err) - }) + engine := NewDefaultInMemForTesting() + defer engine.Close() + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) + if err != nil { + t.Fatal(err) + } + err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) + if err != nil { + t.Fatal(err) + } + _, err = MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil) + if err != nil { + t.Fatal(err) } + + // Delete at a time before the tombstone. Should return a WriteTooOld error. + b := engine.NewBatch() + defer b.Close() + keys, resume, keyCount, err := MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 4}, hlc.ClockTimestamp{}, nil, true) + require.Nil(t, keys) + require.Nil(t, resume) + require.Equal(t, int64(0), keyCount) + require.NotNil(t, err) + require.IsType(t, (*roachpb.WriteTooOldError)(nil), err) + + // Delete at the same time as the tombstone. Should return a WriteTooOld error. + b = engine.NewBatch() + defer b.Close() + keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 5}, hlc.ClockTimestamp{}, nil, true) + require.Nil(t, keys) + require.Nil(t, resume) + require.Equal(t, int64(0), keyCount) + require.NotNil(t, err) + require.IsType(t, (*roachpb.WriteTooOldError)(nil), err) + + // Delete at a time after the tombstone. Should succeed and should not + // include the tombstone in the returned keys. + b = engine.NewBatch() + defer b.Close() + keys, resume, keyCount, err = MVCCDeleteRange(ctx, b, nil, testKey1, testKey4, + math.MaxInt64, hlc.Timestamp{WallTime: 6}, hlc.ClockTimestamp{}, nil, true) + require.Equal(t, []roachpb.Key{testKey1}, keys) + require.Nil(t, resume) + require.Equal(t, int64(1), keyCount) + require.NoError(t, err) } func TestMVCCDeleteRangeInline(t *testing.T) { @@ -1828,108 +1718,104 @@ func TestMVCCDeleteRangeInline(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Make five inline values (zero timestamp). - for i, kv := range []struct { - key roachpb.Key - value roachpb.Value - }{ - {testKey1, value1}, - {testKey2, value2}, - {testKey3, value3}, - {testKey4, value4}, - {testKey5, value5}, - } { - if err := MVCCPut(ctx, engine, nil, kv.key, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, kv.value, nil); err != nil { - t.Fatalf("%d: %+v", i, err) - } - } + // Make five inline values (zero timestamp). + for i, kv := range []struct { + key roachpb.Key + value roachpb.Value + }{ + {testKey1, value1}, + {testKey2, value2}, + {testKey3, value3}, + {testKey4, value4}, + {testKey5, value5}, + } { + if err := MVCCPut(ctx, engine, nil, kv.key, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, kv.value, nil); err != nil { + t.Fatalf("%d: %+v", i, err) + } + } - // Create one non-inline value (non-zero timestamp). - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { - t.Fatal(err) - } + // Create one non-inline value (non-zero timestamp). + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value6, nil); err != nil { + t.Fatal(err) + } - // Attempt to delete two inline keys, should succeed. - deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - 2, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, nil, true) - if err != nil { - t.Fatal(err) - } - if expected := int64(2); num != expected { - t.Fatalf("got %d deleted keys, expected %d", num, expected) - } - if expected := []roachpb.Key{testKey2, testKey3}; !reflect.DeepEqual(deleted, expected) { - t.Fatalf("got deleted values = %v, expected = %v", deleted, expected) - } - if expected := (roachpb.Span{Key: testKey4, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { - t.Fatalf("got resume span = %s, expected = %s", resumeSpan, expected) - } + // Attempt to delete two inline keys, should succeed. + deleted, resumeSpan, num, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, nil, true) + if err != nil { + t.Fatal(err) + } + if expected := int64(2); num != expected { + t.Fatalf("got %d deleted keys, expected %d", num, expected) + } + if expected := []roachpb.Key{testKey2, testKey3}; !reflect.DeepEqual(deleted, expected) { + t.Fatalf("got deleted values = %v, expected = %v", deleted, expected) + } + if expected := (roachpb.Span{Key: testKey4, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { + t.Fatalf("got resume span = %s, expected = %s", resumeSpan, expected) + } - // Attempt to delete inline keys at a timestamp; should fail. - const inlineMismatchErrString = "put is inline" - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey1, testKey6, - 1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true, - ); !testutils.IsError(err, inlineMismatchErrString) { - t.Fatalf("got error %v, expected error with text '%s'", err, inlineMismatchErrString) - } + // Attempt to delete inline keys at a timestamp; should fail. + const inlineMismatchErrString = "put is inline" + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey1, testKey6, + 1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, nil, true, + ); !testutils.IsError(err, inlineMismatchErrString) { + t.Fatalf("got error %v, expected error with text '%s'", err, inlineMismatchErrString) + } - // Attempt to delete non-inline key at zero timestamp; should fail. - const writeTooOldErrString = "WriteTooOldError" - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey6, keyMax, - 1, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, nil, true, - ); !testutils.IsError(err, writeTooOldErrString) { - t.Fatalf("got error %v, expected error with text '%s'", err, writeTooOldErrString) - } + // Attempt to delete non-inline key at zero timestamp; should fail. + const writeTooOldErrString = "WriteTooOldError" + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey6, keyMax, + 1, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, nil, true, + ); !testutils.IsError(err, writeTooOldErrString) { + t.Fatalf("got error %v, expected error with text '%s'", err, writeTooOldErrString) + } - // Attempt to delete inline keys in a transaction; should fail. - if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, - 2, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, txn1, true, - ); !testutils.IsError(err, "writes not allowed within transactions") { - t.Errorf("unexpected error: %+v", err) - } + // Attempt to delete inline keys in a transaction; should fail. + if _, _, _, err := MVCCDeleteRange(ctx, engine, nil, testKey2, testKey6, + 2, hlc.Timestamp{Logical: 0}, hlc.ClockTimestamp{}, txn1, true, + ); !testutils.IsError(err, "writes not allowed within transactions") { + t.Errorf("unexpected error: %+v", err) + } - // Verify final state of the engine. - expectedKvs := []roachpb.KeyValue{ - { - Key: testKey1, - Value: value1, - }, - { - Key: testKey4, - Value: value4, - }, - { - Key: testKey5, - Value: value5, - }, - { - Key: testKey6, - Value: value6, - }, - } - res, err := MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, - MVCCScanOptions{}) - if err != nil { - t.Fatal(err) - } - if a, e := len(res.KVs), len(expectedKvs); a != e { - t.Fatalf("engine scan found %d keys; expected %d", a, e) - } - res.KVs[3].Value.Timestamp = hlc.Timestamp{} - if !reflect.DeepEqual(expectedKvs, res.KVs) { - t.Fatalf( - "engine scan found key/values: %v; expected %v. Diff: %s", - res.KVs, - expectedKvs, - pretty.Diff(res.KVs, expectedKvs), - ) - } - }) + // Verify final state of the engine. + expectedKvs := []roachpb.KeyValue{ + { + Key: testKey1, + Value: value1, + }, + { + Key: testKey4, + Value: value4, + }, + { + Key: testKey5, + Value: value5, + }, + { + Key: testKey6, + Value: value6, + }, + } + res, err := MVCCScan(ctx, engine, localMax, keyMax, hlc.Timestamp{WallTime: 2}, + MVCCScanOptions{}) + if err != nil { + t.Fatal(err) + } + if a, e := len(res.KVs), len(expectedKvs); a != e { + t.Fatalf("engine scan found %d keys; expected %d", a, e) + } + res.KVs[3].Value.Timestamp = hlc.Timestamp{} + if !reflect.DeepEqual(expectedKvs, res.KVs) { + t.Fatalf( + "engine scan found key/values: %v; expected %v. Diff: %s", + res.KVs, + expectedKvs, + pretty.Diff(res.KVs, expectedKvs), + ) } } @@ -2302,90 +2188,86 @@ func TestMVCCInitPut(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - err := MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, false, nil) - if err != nil { - t.Fatal(err) - } + err := MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, false, nil) + if err != nil { + t.Fatal(err) + } - // A repeat of the command will still succeed - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 2}, hlc.ClockTimestamp{}, value1, false, nil) - if err != nil { - t.Fatal(err) - } + // A repeat of the command will still succeed + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 2}, hlc.ClockTimestamp{}, value1, false, nil) + if err != nil { + t.Fatal(err) + } - // Delete. - _, err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, nil) - if err != nil { - t.Fatal(err) - } + // Delete. + _, err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, nil) + if err != nil { + t.Fatal(err) + } - // Reinserting the value fails if we fail on tombstones. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 4}, hlc.ClockTimestamp{}, value1, true, nil) - if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { - if !bytes.Equal(e.ActualValue.RawBytes, nil) { - t.Fatalf("the value %s in get result is not a tombstone", e.ActualValue.RawBytes) - } - } else if err == nil { - t.Fatal("MVCCInitPut with a different value did not fail") - } else { - t.Fatalf("unexpected error %T", e) - } + // Reinserting the value fails if we fail on tombstones. + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 4}, hlc.ClockTimestamp{}, value1, true, nil) + if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { + if !bytes.Equal(e.ActualValue.RawBytes, nil) { + t.Fatalf("the value %s in get result is not a tombstone", e.ActualValue.RawBytes) + } + } else if err == nil { + t.Fatal("MVCCInitPut with a different value did not fail") + } else { + t.Fatalf("unexpected error %T", e) + } - // But doesn't if we *don't* fail on tombstones. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 5}, hlc.ClockTimestamp{}, value1, false, nil) - if err != nil { - t.Fatal(err) - } + // But doesn't if we *don't* fail on tombstones. + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 5}, hlc.ClockTimestamp{}, value1, false, nil) + if err != nil { + t.Fatal(err) + } - // A repeat of the command with a different value will fail. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 6}, hlc.ClockTimestamp{}, value2, false, nil) - if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { - if !bytes.Equal(e.ActualValue.RawBytes, value1.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - e.ActualValue.RawBytes, value1.RawBytes) - } - } else if err == nil { - t.Fatal("MVCCInitPut with a different value did not fail") - } else { - t.Fatalf("unexpected error %T", e) - } + // A repeat of the command with a different value will fail. + err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 6}, hlc.ClockTimestamp{}, value2, false, nil) + if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { + if !bytes.Equal(e.ActualValue.RawBytes, value1.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + e.ActualValue.RawBytes, value1.RawBytes) + } + } else if err == nil { + t.Fatal("MVCCInitPut with a different value did not fail") + } else { + t.Fatalf("unexpected error %T", e) + } - // Ensure that the timestamps were correctly updated. - for _, check := range []struct { - ts, expTS hlc.Timestamp - }{ - {ts: hlc.Timestamp{Logical: 1}, expTS: hlc.Timestamp{Logical: 1}}, - {ts: hlc.Timestamp{Logical: 2}, expTS: hlc.Timestamp{Logical: 2}}, - // If we're checking the future wall time case, the rewrite after delete - // will be present. - {ts: hlc.Timestamp{WallTime: 1}, expTS: hlc.Timestamp{Logical: 5}}, - } { - value, _, err := MVCCGet(ctx, engine, testKey1, check.ts, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - if value.Timestamp != check.expTS { - t.Errorf("value at timestamp %s seen, expected %s", value.Timestamp, check.expTS) - } - } + // Ensure that the timestamps were correctly updated. + for _, check := range []struct { + ts, expTS hlc.Timestamp + }{ + {ts: hlc.Timestamp{Logical: 1}, expTS: hlc.Timestamp{Logical: 1}}, + {ts: hlc.Timestamp{Logical: 2}, expTS: hlc.Timestamp{Logical: 2}}, + // If we're checking the future wall time case, the rewrite after delete + // will be present. + {ts: hlc.Timestamp{WallTime: 1}, expTS: hlc.Timestamp{Logical: 5}}, + } { + value, _, err := MVCCGet(ctx, engine, testKey1, check.ts, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value1.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value1.RawBytes, value.RawBytes) + } + if value.Timestamp != check.expTS { + t.Errorf("value at timestamp %s seen, expected %s", value.Timestamp, check.expTS) + } + } - value, _, pErr := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 0}, MVCCGetOptions{}) - if pErr != nil { - t.Fatal(pErr) - } - if value != nil { - t.Fatalf("%v present at old timestamp", value) - } - }) + value, _, pErr := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 0}, MVCCGetOptions{}) + if pErr != nil { + t.Fatal(pErr) + } + if value != nil { + t.Fatalf("%v present at old timestamp", value) } } @@ -2394,56 +2276,52 @@ func TestMVCCInitPutWithTxn(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - clock := hlc.NewClock(timeutil.NewManualTime(timeutil.Unix(0, 123)), time.Nanosecond /* maxOffset */) + clock := hlc.NewClock(timeutil.NewManualTime(timeutil.Unix(0, 123)), time.Nanosecond /* maxOffset */) - txn := *txn1 - txn.Sequence++ - err := MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, false, &txn) - if err != nil { - t.Fatal(err) - } + txn := *txn1 + txn.Sequence++ + err := MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, false, &txn) + if err != nil { + t.Fatal(err) + } - // A repeat of the command will still succeed. - txn.Sequence++ - err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, false, &txn) - if err != nil { - t.Fatal(err) - } + // A repeat of the command will still succeed. + txn.Sequence++ + err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, false, &txn) + if err != nil { + t.Fatal(err) + } - // A repeat of the command with a different value at a different epoch - // will still succeed. - txn.Sequence++ - txn.Epoch = 2 - err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, false, &txn) - if err != nil { - t.Fatal(err) - } + // A repeat of the command with a different value at a different epoch + // will still succeed. + txn.Sequence++ + txn.Epoch = 2 + err = MVCCInitPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, false, &txn) + if err != nil { + t.Fatal(err) + } - // Commit value3. - txnCommit := txn - txnCommit.Status = roachpb.COMMITTED - txnCommit.WriteTimestamp = clock.Now().Add(1, 0) - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(&txnCommit, roachpb.Span{Key: testKey1})); err != nil { - t.Fatal(err) - } + // Commit value3. + txnCommit := txn + txnCommit.Status = roachpb.COMMITTED + txnCommit.WriteTimestamp = clock.Now().Add(1, 0) + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(&txnCommit, roachpb.Span{Key: testKey1})); err != nil { + t.Fatal(err) + } - // Write value4 with an old timestamp without txn...should get an error. - err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), hlc.ClockTimestamp{}, value4, false, nil) - if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { - if !bytes.Equal(e.ActualValue.RawBytes, value2.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - e.ActualValue.RawBytes, value2.RawBytes) - } - } else { - t.Fatalf("unexpected error %T", e) - } - }) + // Write value4 with an old timestamp without txn...should get an error. + err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), hlc.ClockTimestamp{}, value4, false, nil) + if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { + if !bytes.Equal(e.ActualValue.RawBytes, value2.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + e.ActualValue.RawBytes, value2.RawBytes) + } + } else { + t.Fatalf("unexpected error %T", e) } } @@ -2454,122 +2332,118 @@ func TestMVCCReverseScan(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value4, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value5, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value6, nil); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value3, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value4, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value5, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value6, nil); err != nil { + t.Fatal(err) + } - res, err := MVCCScan(ctx, engine, testKey2, testKey4, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) + res, err := MVCCScan(ctx, engine, testKey2, testKey4, + hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 2 || - !bytes.Equal(res.KVs[0].Key, testKey3) || - !bytes.Equal(res.KVs[1].Key, testKey2) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) || - !bytes.Equal(res.KVs[1].Value.RawBytes, value3.RawBytes) { - t.Fatalf("unexpected value: %v", res.KVs) - } - if res.ResumeSpan != nil { - t.Fatalf("resumeSpan = %+v", res.ResumeSpan) - } + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 2 || + !bytes.Equal(res.KVs[0].Key, testKey3) || + !bytes.Equal(res.KVs[1].Key, testKey2) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) || + !bytes.Equal(res.KVs[1].Value.RawBytes, value3.RawBytes) { + t.Fatalf("unexpected value: %v", res.KVs) + } + if res.ResumeSpan != nil { + t.Fatalf("resumeSpan = %+v", res.ResumeSpan) + } - res, err = MVCCScan(ctx, engine, testKey2, testKey4, hlc.Timestamp{WallTime: 1}, - MVCCScanOptions{Reverse: true, MaxKeys: 1}) + res, err = MVCCScan(ctx, engine, testKey2, testKey4, hlc.Timestamp{WallTime: 1}, + MVCCScanOptions{Reverse: true, MaxKeys: 1}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 || - !bytes.Equal(res.KVs[0].Key, testKey3) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) { - t.Fatalf("unexpected value: %v", res.KVs) - } - if expected := (roachpb.Span{Key: testKey2, EndKey: testKey2.Next()}); !res.ResumeSpan.EqualValue(expected) { - t.Fatalf("expected = %+v, resumeSpan = %+v", expected, res.ResumeSpan) - } + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 || + !bytes.Equal(res.KVs[0].Key, testKey3) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) { + t.Fatalf("unexpected value: %v", res.KVs) + } + if expected := (roachpb.Span{Key: testKey2, EndKey: testKey2.Next()}); !res.ResumeSpan.EqualValue(expected) { + t.Fatalf("expected = %+v, resumeSpan = %+v", expected, res.ResumeSpan) + } - res, err = MVCCScan(ctx, engine, testKey2, testKey4, hlc.Timestamp{WallTime: 1}, - MVCCScanOptions{Reverse: true, MaxKeys: -1}) + res, err = MVCCScan(ctx, engine, testKey2, testKey4, hlc.Timestamp{WallTime: 1}, + MVCCScanOptions{Reverse: true, MaxKeys: -1}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 0 { - t.Fatalf("unexpected value: %v", res.KVs) - } - if expected := (roachpb.Span{Key: testKey2, EndKey: testKey4}); !res.ResumeSpan.EqualValue(expected) { - t.Fatalf("expected = %+v, resumeSpan = %+v", expected, res.ResumeSpan) - } + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 0 { + t.Fatalf("unexpected value: %v", res.KVs) + } + if expected := (roachpb.Span{Key: testKey2, EndKey: testKey4}); !res.ResumeSpan.EqualValue(expected) { + t.Fatalf("expected = %+v, resumeSpan = %+v", expected, res.ResumeSpan) + } - // The first key we encounter has multiple versions and we need to read the - // latest. - res, err = MVCCScan(ctx, engine, testKey2, testKey3, hlc.Timestamp{WallTime: 4}, - MVCCScanOptions{Reverse: true, MaxKeys: 1}) + // The first key we encounter has multiple versions and we need to read the + // latest. + res, err = MVCCScan(ctx, engine, testKey2, testKey3, hlc.Timestamp{WallTime: 4}, + MVCCScanOptions{Reverse: true, MaxKeys: 1}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 || - !bytes.Equal(res.KVs[0].Key, testKey2) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value4.RawBytes) { - t.Errorf("unexpected value: %v", res.KVs) - } + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 || + !bytes.Equal(res.KVs[0].Key, testKey2) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value4.RawBytes) { + t.Errorf("unexpected value: %v", res.KVs) + } - // The first key we encounter is newer than our read timestamp and we need to - // back up to the previous key. - res, err = MVCCScan(ctx, engine, testKey4, testKey6, hlc.Timestamp{WallTime: 1}, - MVCCScanOptions{Reverse: true, MaxKeys: 1}) + // The first key we encounter is newer than our read timestamp and we need to + // back up to the previous key. + res, err = MVCCScan(ctx, engine, testKey4, testKey6, hlc.Timestamp{WallTime: 1}, + MVCCScanOptions{Reverse: true, MaxKeys: 1}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 || - !bytes.Equal(res.KVs[0].Key, testKey4) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) { - t.Fatalf("unexpected value: %v", res.KVs) - } + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 || + !bytes.Equal(res.KVs[0].Key, testKey4) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) { + t.Fatalf("unexpected value: %v", res.KVs) + } - // Scan only the first key in the key space. - res, err = MVCCScan(ctx, engine, testKey1, testKey1.Next(), hlc.Timestamp{WallTime: 1}, - MVCCScanOptions{Reverse: true, MaxKeys: 1}) + // Scan only the first key in the key space. + res, err = MVCCScan(ctx, engine, testKey1, testKey1.Next(), hlc.Timestamp{WallTime: 1}, + MVCCScanOptions{Reverse: true, MaxKeys: 1}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 || - !bytes.Equal(res.KVs[0].Key, testKey1) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) { - t.Fatalf("unexpected value: %v", res.KVs) - } - }) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 || + !bytes.Equal(res.KVs[0].Key, testKey1) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) { + t.Fatalf("unexpected value: %v", res.KVs) } } @@ -2581,38 +2455,34 @@ func TestMVCCReverseScanFirstKeyInFuture(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // The value at key2 will be at a lower timestamp than the ReverseScan, but - // the value at key3 will be at a larger timestamp. The ReverseScan should - // see key3 and ignore it because none of it versions are at a low enough - // timestamp to read. It should then continue scanning backwards and find a - // value at key2. - // - // Before fixing #17825, the MVCC version scan on key3 would fall out of the - // scan bounds and if it never found another valid key before reaching - // KeyMax, would stop the ReverseScan from continuing. - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { - t.Fatal(err) - } + // The value at key2 will be at a lower timestamp than the ReverseScan, but + // the value at key3 will be at a larger timestamp. The ReverseScan should + // see key3 and ignore it because none of it versions are at a low enough + // timestamp to read. It should then continue scanning backwards and find a + // value at key2. + // + // Before fixing #17825, the MVCC version scan on key3 would fall out of the + // scan bounds and if it never found another valid key before reaching + // KeyMax, would stop the ReverseScan from continuing. + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { + t.Fatal(err) + } - res, err := MVCCScan(ctx, engine, testKey1, testKey4, - hlc.Timestamp{WallTime: 2}, MVCCScanOptions{Reverse: true}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 || - !bytes.Equal(res.KVs[0].Key, testKey2) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) { - t.Errorf("unexpected value: %v", res.KVs) - } - }) + res, err := MVCCScan(ctx, engine, testKey1, testKey4, + hlc.Timestamp{WallTime: 2}, MVCCScanOptions{Reverse: true}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 || + !bytes.Equal(res.KVs[0].Key, testKey2) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) { + t.Errorf("unexpected value: %v", res.KVs) } } @@ -2626,35 +2496,31 @@ func TestMVCCReverseScanSeeksOverRepeatedKeys(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // 10 is the value of `kMaxItersBeforeSeek` at the time this test case was - // written. Repeat the key enough times to make sure the `SeekForPrev()` - // optimization will be used. - for i := 1; i <= 10; i++ { - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: int64(i)}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - } - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 11}) - if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { - t.Fatal(err) - } + // 10 is the value of `kMaxItersBeforeSeek` at the time this test case was + // written. Repeat the key enough times to make sure the `SeekForPrev()` + // optimization will be used. + for i := 1; i <= 10; i++ { + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: int64(i)}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + } + txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 11}) + if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + t.Fatal(err) + } - res, err := MVCCScan(ctx, engine, testKey1, testKey3, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 || - !bytes.Equal(res.KVs[0].Key, testKey2) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) { - t.Fatal("unexpected scan results") - } - }) + res, err := MVCCScan(ctx, engine, testKey1, testKey3, + hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 || + !bytes.Equal(res.KVs[0].Key, testKey2) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value2.RawBytes) { + t.Fatal("unexpected scan results") } } @@ -2681,42 +2547,38 @@ func TestMVCCReverseScanSeeksOverRepeatedKeys(t *testing.T) { func TestMVCCReverseScanStopAtSmallestKey(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - run := func(numPuts int, ts int64) { - ctx := context.Background() - engine := engineImpl.create() - defer engine.Close() - - for i := 1; i <= numPuts; i++ { - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: int64(i)}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - } + run := func(numPuts int, ts int64) { + ctx := context.Background() + engine := NewDefaultInMemForTesting() + defer engine.Close() - res, err := MVCCScan(ctx, engine, testKey1, testKey3, - hlc.Timestamp{WallTime: ts}, MVCCScanOptions{Reverse: true}) - if err != nil { - t.Fatal(err) - } - if len(res.KVs) != 1 || - !bytes.Equal(res.KVs[0].Key, testKey1) || - !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) { - t.Fatal("unexpected scan results") - } - } - // Satisfying (2) and (3) is incredibly intricate because of how `iters_before_seek_` - // is incremented/decremented heuristically. For example, at the time of writing, the - // infinitely looping cases are `numPuts == 6 && ts == 2`, `numPuts == 7 && ts == 3`, - // `numPuts == 8 && ts == 4`, `numPuts == 9 && ts == 5`, and `numPuts == 10 && ts == 6`. - // Tying our test case to the `iters_before_seek_` setting logic seems brittle so let's - // just brute force test a wide range of cases. - for numPuts := 1; numPuts <= 10; numPuts++ { - for ts := 1; ts <= 10; ts++ { - run(numPuts, int64(ts)) - } + for i := 1; i <= numPuts; i++ { + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: int64(i)}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) } - }) + } + + res, err := MVCCScan(ctx, engine, testKey1, testKey3, + hlc.Timestamp{WallTime: ts}, MVCCScanOptions{Reverse: true}) + if err != nil { + t.Fatal(err) + } + if len(res.KVs) != 1 || + !bytes.Equal(res.KVs[0].Key, testKey1) || + !bytes.Equal(res.KVs[0].Value.RawBytes, value1.RawBytes) { + t.Fatal("unexpected scan results") + } + } + // Satisfying (2) and (3) is incredibly intricate because of how `iters_before_seek_` + // is incremented/decremented heuristically. For example, at the time of writing, the + // infinitely looping cases are `numPuts == 6 && ts == 2`, `numPuts == 7 && ts == 3`, + // `numPuts == 8 && ts == 4`, `numPuts == 9 && ts == 5`, and `numPuts == 10 && ts == 6`. + // Tying our test case to the `iters_before_seek_` setting logic seems brittle so let's + // just brute force test a wide range of cases. + for numPuts := 1; numPuts <= 10; numPuts++ { + for ts := 1; ts <= 10; ts++ { + run(numPuts, int64(ts)) + } } } @@ -2725,45 +2587,41 @@ func TestMVCCResolveTxn(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + t.Fatal(err) + } - { - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{ - Txn: txn1, - }) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - } + { + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{ + Txn: txn1, + }) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value1.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value1.RawBytes, value.RawBytes) + } + } - // Resolve will write with txn1's timestamp which is 0,1. - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn1Commit, roachpb.Span{Key: testKey1})); err != nil { - t.Fatal(err) - } + // Resolve will write with txn1's timestamp which is 0,1. + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn1Commit, roachpb.Span{Key: testKey1})); err != nil { + t.Fatal(err) + } - { - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - } - }) + { + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value1.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value1.RawBytes, value.RawBytes) + } } } @@ -2774,36 +2632,32 @@ func TestMVCCResolveNewerIntent(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Write first value. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1Commit.WriteTimestamp, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - // Now, put down an intent which should return a write too old error - // (but will still write the intent at tx1Commit.Timestamp+1. - err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1) - if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { - t.Fatalf("expected write too old error; got %s", err) - } + // Write first value. + if err := MVCCPut(ctx, engine, nil, testKey1, txn1Commit.WriteTimestamp, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + // Now, put down an intent which should return a write too old error + // (but will still write the intent at tx1Commit.Timestamp+1. + err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1) + if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { + t.Fatalf("expected write too old error; got %s", err) + } - // Resolve will succeed but should remove the intent. - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn1Commit, roachpb.Span{Key: testKey1})); err != nil { - t.Fatal(err) - } + // Resolve will succeed but should remove the intent. + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn1Commit, roachpb.Span{Key: testKey1})); err != nil { + t.Fatal(err) + } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 2}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("expected value1 bytes; got %q", value.RawBytes) - } - }) + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 2}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value1.RawBytes, value.RawBytes) { + t.Fatalf("expected value1 bytes; got %q", value.RawBytes) } } @@ -2812,49 +2666,45 @@ func TestMVCCResolveIntentTxnTimestampMismatch(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - txn := txn1.Clone() - tsEarly := txn.WriteTimestamp - txn.TxnMeta.WriteTimestamp.Forward(tsEarly.Add(10, 0)) + txn := txn1.Clone() + tsEarly := txn.WriteTimestamp + txn.TxnMeta.WriteTimestamp.Forward(tsEarly.Add(10, 0)) - // Write an intent which has txn.WriteTimestamp > meta.timestamp. - if err := MVCCPut(ctx, engine, nil, testKey1, tsEarly, hlc.ClockTimestamp{}, value1, txn); err != nil { - t.Fatal(err) - } + // Write an intent which has txn.WriteTimestamp > meta.timestamp. + if err := MVCCPut(ctx, engine, nil, testKey1, tsEarly, hlc.ClockTimestamp{}, value1, txn); err != nil { + t.Fatal(err) + } - // The Timestamp within is equal to that of txn.Meta even though - // the intent sits at tsEarly. The bug was looking at the former - // instead of the latter (and so we could also tickle it with - // smaller timestamps in Txn). - intent := roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1}) - intent.Status = roachpb.PENDING + // The Timestamp within is equal to that of txn.Meta even though + // the intent sits at tsEarly. The bug was looking at the former + // instead of the latter (and so we could also tickle it with + // smaller timestamps in Txn). + intent := roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1}) + intent.Status = roachpb.PENDING - // A bug (see #7654) caused intents to just stay where they were instead - // of being moved forward in the situation set up above. - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, intent); err != nil { - t.Fatal(err) - } + // A bug (see #7654) caused intents to just stay where they were instead + // of being moved forward in the situation set up above. + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, intent); err != nil { + t.Fatal(err) + } - for i, test := range []struct { - hlc.Timestamp - found bool - }{ - // Check that the intent has indeed moved to where we pushed it. - {tsEarly, false}, - {intent.Txn.WriteTimestamp.Prev(), false}, - {intent.Txn.WriteTimestamp, true}, - {hlc.MaxTimestamp, true}, - } { - _, _, err := MVCCGet(ctx, engine, testKey1, test.Timestamp, MVCCGetOptions{}) - if errors.HasType(err, (*roachpb.WriteIntentError)(nil)) != test.found { - t.Fatalf("%d: expected write intent error: %t, got %v", i, test.found, err) - } - } - }) + for i, test := range []struct { + hlc.Timestamp + found bool + }{ + // Check that the intent has indeed moved to where we pushed it. + {tsEarly, false}, + {intent.Txn.WriteTimestamp.Prev(), false}, + {intent.Txn.WriteTimestamp, true}, + {hlc.MaxTimestamp, true}, + } { + _, _, err := MVCCGet(ctx, engine, testKey1, test.Timestamp, MVCCGetOptions{}) + if errors.HasType(err, (*roachpb.WriteIntentError)(nil)) != test.found { + t.Fatalf("%d: expected write intent error: %t, got %v", i, test.found, err) + } } } @@ -2866,48 +2716,44 @@ func TestMVCCResolveIntentTxnTimestampMismatch(t *testing.T) { // WriteTooOldError if that timestamp isn't recent. func TestMVCCConditionalPutOldTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) - if err != nil { - t.Fatal(err) - } - err = MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) - if err != nil { - t.Fatal(err) - } + defer log.Scope(t).Close(t) - // Check nothing is written if the value doesn't match. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value3, value1.TagAndDataBytes(), CPutFailIfMissing, nil) - if err == nil { - t.Errorf("unexpected success on conditional put") - } - if !errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) { - t.Errorf("unexpected error on conditional put: %+v", err) - } + ctx := context.Background() + engine := NewDefaultInMemForTesting() + defer engine.Close() + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil) + if err != nil { + t.Fatal(err) + } + err = MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil) + if err != nil { + t.Fatal(err) + } - // But if value does match the most recently written version, we'll get - // a write too old error but still write updated value. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value3, value2.TagAndDataBytes(), CPutFailIfMissing, nil) - if err == nil { - t.Errorf("unexpected success on conditional put") - } - if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { - t.Errorf("unexpected error on conditional put: %+v", err) - } - // Verify new value was actually written at (3, 1). - ts := hlc.Timestamp{WallTime: 3, Logical: 1} - value, _, err := MVCCGet(ctx, engine, testKey1, ts, MVCCGetOptions{}) - if err != nil || value.Timestamp != ts || !bytes.Equal(value3.RawBytes, value.RawBytes) { - t.Fatalf("expected err=nil (got %s), timestamp=%s (got %s), value=%q (got %q)", - err, value.Timestamp, ts, value3.RawBytes, value.RawBytes) - } - }) + // Check nothing is written if the value doesn't match. + err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value3, value1.TagAndDataBytes(), CPutFailIfMissing, nil) + if err == nil { + t.Errorf("unexpected success on conditional put") + } + if !errors.HasType(err, (*roachpb.ConditionFailedError)(nil)) { + t.Errorf("unexpected error on conditional put: %+v", err) + } + + // But if value does match the most recently written version, we'll get + // a write too old error but still write updated value. + err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, hlc.ClockTimestamp{}, value3, value2.TagAndDataBytes(), CPutFailIfMissing, nil) + if err == nil { + t.Errorf("unexpected success on conditional put") + } + if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { + t.Errorf("unexpected error on conditional put: %+v", err) + } + // Verify new value was actually written at (3, 1). + ts := hlc.Timestamp{WallTime: 3, Logical: 1} + value, _, err := MVCCGet(ctx, engine, testKey1, ts, MVCCGetOptions{}) + if err != nil || value.Timestamp != ts || !bytes.Equal(value3.RawBytes, value.RawBytes) { + t.Fatalf("expected err=nil (got %s), timestamp=%s (got %s), value=%q (got %q)", + err, value.Timestamp, ts, value3.RawBytes, value.RawBytes) } } @@ -2921,52 +2767,48 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil) - if err != nil { - t.Fatal(err) - } + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil) + if err != nil { + t.Fatal(err) + } - // Verify the first txn Put returns a write too old error, but the - // intent is written at the advanced timestamp. - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) - if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { - t.Errorf("expected WriteTooOldError on Put; got %v", err) - } - // Verify new value was actually written at (3, 1). - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) - if err != nil { - t.Fatal(err) - } - expTS := hlc.Timestamp{WallTime: 3, Logical: 1} - if value.Timestamp != expTS || !bytes.Equal(value2.RawBytes, value.RawBytes) { - t.Fatalf("expected timestamp=%s (got %s), value=%q (got %q)", - value.Timestamp, expTS, value2.RawBytes, value.RawBytes) - } + // Verify the first txn Put returns a write too old error, but the + // intent is written at the advanced timestamp. + txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn.Sequence++ + err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) + if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { + t.Errorf("expected WriteTooOldError on Put; got %v", err) + } + // Verify new value was actually written at (3, 1). + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) + if err != nil { + t.Fatal(err) + } + expTS := hlc.Timestamp{WallTime: 3, Logical: 1} + if value.Timestamp != expTS || !bytes.Equal(value2.RawBytes, value.RawBytes) { + t.Fatalf("expected timestamp=%s (got %s), value=%q (got %q)", + value.Timestamp, expTS, value2.RawBytes, value.RawBytes) + } - // Put again and verify no WriteTooOldError, but timestamp should continue - // to be set to (3,1). - txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn) - if err != nil { - t.Error(err) - } - // Verify new value was actually written at (3, 1). - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) - if err != nil { - t.Fatal(err) - } - if value.Timestamp != expTS || !bytes.Equal(value3.RawBytes, value.RawBytes) { - t.Fatalf("expected timestamp=%s (got %s), value=%q (got %q)", - value.Timestamp, expTS, value3.RawBytes, value.RawBytes) - } - }) + // Put again and verify no WriteTooOldError, but timestamp should continue + // to be set to (3,1). + txn.Sequence++ + err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn) + if err != nil { + t.Error(err) + } + // Verify new value was actually written at (3, 1). + value, _, err = MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) + if err != nil { + t.Fatal(err) + } + if value.Timestamp != expTS || !bytes.Equal(value3.RawBytes, value.RawBytes) { + t.Fatalf("expected timestamp=%s (got %s), value=%q (got %q)", + value.Timestamp, expTS, value3.RawBytes, value.RawBytes) } } @@ -2975,19 +2817,13 @@ func TestMVCCPutNegativeTimestampError(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - timestamp := hlc.Timestamp{WallTime: -1} - expectedErrorString := fmt.Sprintf("cannot write to %q at timestamp %s", testKey1, timestamp) - - err := MVCCPut(ctx, engine, nil, testKey1, timestamp, hlc.ClockTimestamp{}, value1, nil) + engine := NewDefaultInMemForTesting() + defer engine.Close() - require.EqualError(t, err, expectedErrorString) - }) - } + timestamp := hlc.Timestamp{WallTime: -1} + expectedErrorString := fmt.Sprintf("cannot write to %q at timestamp %s", testKey1, timestamp) + err := MVCCPut(ctx, engine, nil, testKey1, timestamp, hlc.ClockTimestamp{}, value1, nil) + require.EqualError(t, err, expectedErrorString) } // TestMVCCPutOldOrigTimestampNewCommitTimestamp tests a case where a @@ -3001,42 +2837,38 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil) - if err != nil { - t.Fatal(err) - } + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value1, nil) + if err != nil { + t.Fatal(err) + } - // Perform a transactional Put with a transaction whose original timestamp is - // below the existing key's timestamp and whose provisional commit timestamp - // is above the existing key's timestamp. - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - txn.WriteTimestamp = hlc.Timestamp{WallTime: 5} - txn.Sequence++ - err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) - - // Verify that the Put returned a WriteTooOld with the ActualTime set to the - // transactions provisional commit timestamp. - expTS := txn.WriteTimestamp - if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) || wtoErr.ActualTimestamp != expTS { - t.Fatalf("expected WriteTooOldError with actual time = %s; got %s", expTS, wtoErr) - } + // Perform a transactional Put with a transaction whose original timestamp is + // below the existing key's timestamp and whose provisional commit timestamp + // is above the existing key's timestamp. + txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn.WriteTimestamp = hlc.Timestamp{WallTime: 5} + txn.Sequence++ + err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn) - // Verify new value was actually written at the transaction's provisional - // commit timestamp. - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) - if err != nil { - t.Fatal(err) - } - if value.Timestamp != expTS || !bytes.Equal(value2.RawBytes, value.RawBytes) { - t.Fatalf("expected timestamp=%s (got %s), value=%q (got %q)", - value.Timestamp, expTS, value2.RawBytes, value.RawBytes) - } - }) + // Verify that the Put returned a WriteTooOld with the ActualTime set to the + // transactions provisional commit timestamp. + expTS := txn.WriteTimestamp + if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) || wtoErr.ActualTimestamp != expTS { + t.Fatalf("expected WriteTooOldError with actual time = %s; got %s", expTS, wtoErr) + } + + // Verify new value was actually written at the transaction's provisional + // commit timestamp. + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) + if err != nil { + t.Fatal(err) + } + if value.Timestamp != expTS || !bytes.Equal(value2.RawBytes, value.RawBytes) { + t.Fatalf("expected timestamp=%s (got %s), value=%q (got %q)", + value.Timestamp, expTS, value2.RawBytes, value.RawBytes) } } @@ -3045,34 +2877,30 @@ func TestMVCCAbortTxn(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + t.Fatal(err) + } - txn1AbortWithTS := txn1Abort.Clone() - txn1AbortWithTS.WriteTimestamp = hlc.Timestamp{Logical: 1} + txn1AbortWithTS := txn1Abort.Clone() + txn1AbortWithTS.WriteTimestamp = hlc.Timestamp{Logical: 1} - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn1AbortWithTS, roachpb.Span{Key: testKey1}), - ); err != nil { - t.Fatal(err) - } + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn1AbortWithTS, roachpb.Span{Key: testKey1}), + ); err != nil { + t.Fatal(err) + } - if value, _, err := MVCCGet( - ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}, - ); err != nil { - t.Fatal(err) - } else if value != nil { - t.Fatalf("expected the value to be empty: %s", value) - } - require.Empty(t, mvccGetRaw(t, engine, mvccKey(testKey1))) - }) + if value, _, err := MVCCGet( + ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}, + ); err != nil { + t.Fatal(err) + } else if value != nil { + t.Fatalf("expected the value to be empty: %s", value) } + require.Empty(t, mvccGetRaw(t, engine, mvccKey(testKey1))) } func TestMVCCAbortTxnWithPreviousVersion(t *testing.T) { @@ -3080,50 +2908,46 @@ func TestMVCCAbortTxnWithPreviousVersion(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts); err != nil { + t.Fatal(err) + } - txn1AbortWithTS := txn1Abort.Clone() - txn1AbortWithTS.WriteTimestamp = hlc.Timestamp{WallTime: 2} + txn1AbortWithTS := txn1Abort.Clone() + txn1AbortWithTS.WriteTimestamp = hlc.Timestamp{WallTime: 2} - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn1AbortWithTS, roachpb.Span{Key: testKey1}), - ); err != nil { - t.Fatal(err) - } + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn1AbortWithTS, roachpb.Span{Key: testKey1}), + ); err != nil { + t.Fatal(err) + } - if _, intent, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{ - Inconsistent: true, - }); err != nil { - t.Fatal(err) - } else if intent != nil { - t.Fatalf("expected no intent, got: %s", intent) - } + if _, intent, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{ + Inconsistent: true, + }); err != nil { + t.Fatal(err) + } else if intent != nil { + t.Fatalf("expected no intent, got: %s", intent) + } - if value, _, err := MVCCGet( - ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCGetOptions{}, - ); err != nil { - t.Fatal(err) - } else if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { - t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) - } else if !bytes.Equal(value2.RawBytes, value.RawBytes) { - t.Fatalf("the value %q in get result does not match the value %q in request", - value.RawBytes, value2.RawBytes) - } - }) + if value, _, err := MVCCGet( + ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCGetOptions{}, + ); err != nil { + t.Fatal(err) + } else if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { + t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) + } else if !bytes.Equal(value2.RawBytes, value.RawBytes) { + t.Fatalf("the value %q in get result does not match the value %q in request", + value.RawBytes, value2.RawBytes) } } @@ -3132,96 +2956,92 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - // Start with epoch 1. - txn := *txn1 - txn.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txn); err != nil { - t.Fatal(err) - } - // Now write with greater timestamp and epoch 2. - txne2 := txn - txne2.Sequence++ - txne2.Epoch = 2 - txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value2, &txne2); err != nil { - t.Fatal(err) - } - // Try a write with an earlier timestamp; this is just ignored. - txne2.Sequence++ - txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txne2); err != nil { - t.Fatal(err) - } - // Try a write with an earlier epoch; again ignored. - if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txn); err == nil { - t.Fatal("unexpected success of a write with an earlier epoch") - } - // Try a write with different value using both later timestamp and epoch. - txne2.Sequence++ - if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value3, &txne2); err != nil { - t.Fatal(err) - } - // Resolve the intent. - txne2Commit := txne2 - txne2Commit.Status = roachpb.COMMITTED - txne2Commit.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(&txne2Commit, roachpb.Span{Key: testKey1})); err != nil { - t.Fatal(err) - } + engine := NewDefaultInMemForTesting() + defer engine.Close() - expTS := txne2Commit.WriteTimestamp.Next() + // Start with epoch 1. + txn := *txn1 + txn.Sequence++ + if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txn); err != nil { + t.Fatal(err) + } + // Now write with greater timestamp and epoch 2. + txne2 := txn + txne2.Sequence++ + txne2.Epoch = 2 + txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} + if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value2, &txne2); err != nil { + t.Fatal(err) + } + // Try a write with an earlier timestamp; this is just ignored. + txne2.Sequence++ + txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} + if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txne2); err != nil { + t.Fatal(err) + } + // Try a write with an earlier epoch; again ignored. + if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, hlc.ClockTimestamp{}, value1, &txn); err == nil { + t.Fatal("unexpected success of a write with an earlier epoch") + } + // Try a write with different value using both later timestamp and epoch. + txne2.Sequence++ + if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, hlc.ClockTimestamp{}, value3, &txne2); err != nil { + t.Fatal(err) + } + // Resolve the intent. + txne2Commit := txne2 + txne2Commit.Status = roachpb.COMMITTED + txne2Commit.WriteTimestamp = hlc.Timestamp{WallTime: 1} + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(&txne2Commit, roachpb.Span{Key: testKey1})); err != nil { + t.Fatal(err) + } - // Now try writing an earlier value without a txn--should get WriteTooOldError. - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value4, nil) - if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { - t.Fatal("unexpected success") - } else if wtoErr.ActualTimestamp != expTS { - t.Fatalf("expected write too old error with actual ts %s; got %s", expTS, wtoErr.ActualTimestamp) - } - // Verify value was actually written at (1, 1). - value, _, err := MVCCGet(ctx, engine, testKey1, expTS, MVCCGetOptions{}) - if err != nil || value.Timestamp != expTS || !bytes.Equal(value4.RawBytes, value.RawBytes) { - t.Fatalf("expected err=nil (got %s), timestamp=%s (got %s), value=%q (got %q)", - err, value.Timestamp, expTS, value4.RawBytes, value.RawBytes) - } - // Now write an intent with exactly the same timestamp--ties also get WriteTooOldError. - err = MVCCPut(ctx, engine, nil, testKey1, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value5, txn2) - intentTS := expTS.Next() - if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { - t.Fatal("unexpected success") - } else if wtoErr.ActualTimestamp != intentTS { - t.Fatalf("expected write too old error with actual ts %s; got %s", intentTS, wtoErr.ActualTimestamp) - } - // Verify intent value was actually written at (1, 2). - value, _, err = MVCCGet(ctx, engine, testKey1, intentTS, MVCCGetOptions{Txn: txn2}) - if err != nil || value.Timestamp != intentTS || !bytes.Equal(value5.RawBytes, value.RawBytes) { - t.Fatalf("expected err=nil (got %s), timestamp=%s (got %s), value=%q (got %q)", - err, value.Timestamp, intentTS, value5.RawBytes, value.RawBytes) - } - // Attempt to read older timestamp; should fail. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 0}, MVCCGetOptions{}) - if value != nil || err != nil { - t.Fatalf("expected value nil, err nil; got %+v, %v", value, err) - } - // Read at correct timestamp. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { - t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) - } - if !bytes.Equal(value3.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value3.RawBytes, value.RawBytes) - } - }) + expTS := txne2Commit.WriteTimestamp.Next() + + // Now try writing an earlier value without a txn--should get WriteTooOldError. + err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value4, nil) + if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { + t.Fatal("unexpected success") + } else if wtoErr.ActualTimestamp != expTS { + t.Fatalf("expected write too old error with actual ts %s; got %s", expTS, wtoErr.ActualTimestamp) + } + // Verify value was actually written at (1, 1). + value, _, err := MVCCGet(ctx, engine, testKey1, expTS, MVCCGetOptions{}) + if err != nil || value.Timestamp != expTS || !bytes.Equal(value4.RawBytes, value.RawBytes) { + t.Fatalf("expected err=nil (got %s), timestamp=%s (got %s), value=%q (got %q)", + err, value.Timestamp, expTS, value4.RawBytes, value.RawBytes) + } + // Now write an intent with exactly the same timestamp--ties also get WriteTooOldError. + err = MVCCPut(ctx, engine, nil, testKey1, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value5, txn2) + intentTS := expTS.Next() + if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { + t.Fatal("unexpected success") + } else if wtoErr.ActualTimestamp != intentTS { + t.Fatalf("expected write too old error with actual ts %s; got %s", intentTS, wtoErr.ActualTimestamp) + } + // Verify intent value was actually written at (1, 2). + value, _, err = MVCCGet(ctx, engine, testKey1, intentTS, MVCCGetOptions{Txn: txn2}) + if err != nil || value.Timestamp != intentTS || !bytes.Equal(value5.RawBytes, value.RawBytes) { + t.Fatalf("expected err=nil (got %s), timestamp=%s (got %s), value=%q (got %q)", + err, value.Timestamp, intentTS, value5.RawBytes, value.RawBytes) + } + // Attempt to read older timestamp; should fail. + value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 0}, MVCCGetOptions{}) + if value != nil || err != nil { + t.Fatalf("expected value nil, err nil; got %+v, %v", value, err) + } + // Read at correct timestamp. + value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { + t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) + } + if !bytes.Equal(value3.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value3.RawBytes, value.RawBytes) } } @@ -3232,51 +3052,47 @@ func TestMVCCGetWithDiffEpochs(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - ctx := context.Background() - engine := engineImpl.create() - defer engine.Close() + ctx := context.Background() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Write initial value without a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - // Now write using txn1, epoch 1. - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { - t.Fatal(err) - } - // Try reading using different txns & epochs. - testCases := []struct { - txn *roachpb.Transaction - expValue *roachpb.Value - expErr bool - }{ - // No transaction; should see error. - {nil, nil, true}, - // Txn1, epoch 1; should see new value2. - {txn1, &value2, false}, - // Txn1, epoch 2; should see original value1. - {txn1e2, &value1, false}, - // Txn2; should see error. - {txn2, nil, true}, - } - for i, test := range testCases { - t.Run(strconv.Itoa(i), func(t *testing.T) { - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ - Txn: test.txn, - }) - if test.expErr { - if err == nil { - t.Errorf("test %d: unexpected success", i) - } else if !errors.HasType(err, (*roachpb.WriteIntentError)(nil)) { - t.Errorf("test %d: expected write intent error; got %v", i, err) - } - } else if err != nil || value == nil || !bytes.Equal(test.expValue.RawBytes, value.RawBytes) { - t.Errorf("test %d: expected value %q, err nil; got %+v, %v", i, test.expValue.RawBytes, value, err) - } - }) + // Write initial value without a txn. + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + // Now write using txn1, epoch 1. + txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1ts); err != nil { + t.Fatal(err) + } + // Try reading using different txns & epochs. + testCases := []struct { + txn *roachpb.Transaction + expValue *roachpb.Value + expErr bool + }{ + // No transaction; should see error. + {nil, nil, true}, + // Txn1, epoch 1; should see new value2. + {txn1, &value2, false}, + // Txn1, epoch 2; should see original value1. + {txn1e2, &value1, false}, + // Txn2; should see error. + {txn2, nil, true}, + } + for i, test := range testCases { + t.Run(strconv.Itoa(i), func(t *testing.T) { + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ + Txn: test.txn, + }) + if test.expErr { + if err == nil { + t.Errorf("test %d: unexpected success", i) + } else if !errors.HasType(err, (*roachpb.WriteIntentError)(nil)) { + t.Errorf("test %d: expected write intent error; got %v", i, err) + } + } else if err != nil || value == nil || !bytes.Equal(test.expValue.RawBytes, value.RawBytes) { + t.Errorf("test %d: expected value %q, err nil; got %+v, %v", i, test.expValue.RawBytes, value, err) } }) } @@ -3293,65 +3109,61 @@ func TestMVCCGetWithDiffEpochsAndTimestamps(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - ctx := context.Background() - engine := engineImpl.create() - defer engine.Close() + ctx := context.Background() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Write initial value without a txn at timestamp 1. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - // Write another value without a txn at timestamp 3. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - // Now write using txn1, epoch 1. - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - // Bump epoch 1's write timestamp to timestamp 4. - txn1ts.WriteTimestamp = hlc.Timestamp{WallTime: 4} - // Expected to hit WriteTooOld error but to still lay down intent. - err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts) - if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { - t.Fatalf("unexpectedly not WriteTooOld: %+v", err) - } else if expTS, actTS := txn1ts.WriteTimestamp, wtoErr.ActualTimestamp; expTS != actTS { - t.Fatalf("expected write too old error with actual ts %s; got %s", expTS, actTS) - } - // Try reading using different epochs & timestamps. - testCases := []struct { - txn *roachpb.Transaction - readTS hlc.Timestamp - expValue *roachpb.Value - }{ - // Epoch 1, read 1; should see new value3. - {txn1, hlc.Timestamp{WallTime: 1}, &value3}, - // Epoch 1, read 2; should see new value3. - {txn1, hlc.Timestamp{WallTime: 2}, &value3}, - // Epoch 1, read 3; should see new value3. - {txn1, hlc.Timestamp{WallTime: 3}, &value3}, - // Epoch 1, read 4; should see new value3. - {txn1, hlc.Timestamp{WallTime: 4}, &value3}, - // Epoch 1, read 5; should see new value3. - {txn1, hlc.Timestamp{WallTime: 5}, &value3}, - // Epoch 2, read 1; should see committed value1. - {txn1e2, hlc.Timestamp{WallTime: 1}, &value1}, - // Epoch 2, read 2; should see committed value1. - {txn1e2, hlc.Timestamp{WallTime: 2}, &value1}, - // Epoch 2, read 3; should see committed value2. - {txn1e2, hlc.Timestamp{WallTime: 3}, &value2}, - // Epoch 2, read 4; should see committed value2. - {txn1e2, hlc.Timestamp{WallTime: 4}, &value2}, - // Epoch 2, read 5; should see committed value2. - {txn1e2, hlc.Timestamp{WallTime: 5}, &value2}, - } - for i, test := range testCases { - t.Run(strconv.Itoa(i), func(t *testing.T) { - value, _, err := MVCCGet(ctx, engine, testKey1, test.readTS, MVCCGetOptions{Txn: test.txn}) - if err != nil || value == nil || !bytes.Equal(test.expValue.RawBytes, value.RawBytes) { - t.Errorf("test %d: expected value %q, err nil; got %+v, %v", i, test.expValue.RawBytes, value, err) - } - }) + // Write initial value without a txn at timestamp 1. + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + // Write another value without a txn at timestamp 3. + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + // Now write using txn1, epoch 1. + txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + // Bump epoch 1's write timestamp to timestamp 4. + txn1ts.WriteTimestamp = hlc.Timestamp{WallTime: 4} + // Expected to hit WriteTooOld error but to still lay down intent. + err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn1ts) + if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { + t.Fatalf("unexpectedly not WriteTooOld: %+v", err) + } else if expTS, actTS := txn1ts.WriteTimestamp, wtoErr.ActualTimestamp; expTS != actTS { + t.Fatalf("expected write too old error with actual ts %s; got %s", expTS, actTS) + } + // Try reading using different epochs & timestamps. + testCases := []struct { + txn *roachpb.Transaction + readTS hlc.Timestamp + expValue *roachpb.Value + }{ + // Epoch 1, read 1; should see new value3. + {txn1, hlc.Timestamp{WallTime: 1}, &value3}, + // Epoch 1, read 2; should see new value3. + {txn1, hlc.Timestamp{WallTime: 2}, &value3}, + // Epoch 1, read 3; should see new value3. + {txn1, hlc.Timestamp{WallTime: 3}, &value3}, + // Epoch 1, read 4; should see new value3. + {txn1, hlc.Timestamp{WallTime: 4}, &value3}, + // Epoch 1, read 5; should see new value3. + {txn1, hlc.Timestamp{WallTime: 5}, &value3}, + // Epoch 2, read 1; should see committed value1. + {txn1e2, hlc.Timestamp{WallTime: 1}, &value1}, + // Epoch 2, read 2; should see committed value1. + {txn1e2, hlc.Timestamp{WallTime: 2}, &value1}, + // Epoch 2, read 3; should see committed value2. + {txn1e2, hlc.Timestamp{WallTime: 3}, &value2}, + // Epoch 2, read 4; should see committed value2. + {txn1e2, hlc.Timestamp{WallTime: 4}, &value2}, + // Epoch 2, read 5; should see committed value2. + {txn1e2, hlc.Timestamp{WallTime: 5}, &value2}, + } + for i, test := range testCases { + t.Run(strconv.Itoa(i), func(t *testing.T) { + value, _, err := MVCCGet(ctx, engine, testKey1, test.readTS, MVCCGetOptions{Txn: test.txn}) + if err != nil || value == nil || !bytes.Equal(test.expValue.RawBytes, value.RawBytes) { + t.Errorf("test %d: expected value %q, err nil; got %+v, %v", i, test.expValue.RawBytes, value, err) } }) } @@ -3363,22 +3175,18 @@ func TestMVCCGetWithOldEpoch(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - ctx := context.Background() - engine := engineImpl.create() - defer engine.Close() + ctx := context.Background() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1e2); err != nil { - t.Fatal(err) - } - _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ - Txn: txn1, - }) - if err == nil { - t.Fatalf("unexpected success of get") - } - }) + if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1e2); err != nil { + t.Fatal(err) + } + _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ + Txn: txn1, + }) + if err == nil { + t.Fatalf("unexpected success of get") } } @@ -3392,65 +3200,61 @@ func TestMVCCDeleteRangeWithSequence(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - testCases := []struct { - name string - sequence enginepb.TxnSeq - expErr string - }{ - {"old seq", 5, "missing an intent"}, - {"same seq", 6, ""}, - {"new seq", 7, ""}, - } + testCases := []struct { + name string + sequence enginepb.TxnSeq + expErr string + }{ + {"old seq", 5, "missing an intent"}, + {"same seq", 6, ""}, + {"new seq", 7, ""}, + } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - prefix := roachpb.Key(fmt.Sprintf("key-%d", tc.sequence)) - txn := *txn1 - for i := enginepb.TxnSeq(0); i < 3; i++ { - key := append(prefix, []byte(strconv.Itoa(int(i)))...) - txn.Sequence = 2 + i - if err := MVCCPut(ctx, engine, nil, key, txn.WriteTimestamp, hlc.ClockTimestamp{}, value1, &txn); err != nil { - t.Fatal(err) - } - } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + prefix := roachpb.Key(fmt.Sprintf("key-%d", tc.sequence)) + txn := *txn1 + for i := enginepb.TxnSeq(0); i < 3; i++ { + key := append(prefix, []byte(strconv.Itoa(int(i)))...) + txn.Sequence = 2 + i + if err := MVCCPut(ctx, engine, nil, key, txn.WriteTimestamp, hlc.ClockTimestamp{}, value1, &txn); err != nil { + t.Fatal(err) + } + } - // Perform the initial DeleteRange. - const origSeq = 6 - txn.Sequence = origSeq - origDeleted, _, origNum, err := MVCCDeleteRange(ctx, engine, nil, - prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, hlc.ClockTimestamp{}, &txn, true) - if err != nil { - t.Fatal(err) - } + // Perform the initial DeleteRange. + const origSeq = 6 + txn.Sequence = origSeq + origDeleted, _, origNum, err := MVCCDeleteRange(ctx, engine, nil, + prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, hlc.ClockTimestamp{}, &txn, true) + if err != nil { + t.Fatal(err) + } - txn.Sequence = tc.sequence - deleted, _, num, err := MVCCDeleteRange(ctx, engine, nil, - prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, hlc.ClockTimestamp{}, &txn, true) - if tc.expErr != "" && err != nil { - if !testutils.IsError(err, tc.expErr) { - t.Fatalf("unexpected error: %+v", err) - } - } else if err != nil { - t.Fatalf("unexpected error: %+v", err) - } + txn.Sequence = tc.sequence + deleted, _, num, err := MVCCDeleteRange(ctx, engine, nil, + prefix, prefix.PrefixEnd(), math.MaxInt64, txn.WriteTimestamp, hlc.ClockTimestamp{}, &txn, true) + if tc.expErr != "" && err != nil { + if !testutils.IsError(err, tc.expErr) { + t.Fatalf("unexpected error: %+v", err) + } + } else if err != nil { + t.Fatalf("unexpected error: %+v", err) + } - // If at the same sequence as the initial DeleteRange. - if tc.sequence == origSeq { - if !reflect.DeepEqual(origDeleted, deleted) { - t.Fatalf("deleted keys did not match original execution: %+v vs. %+v", - origDeleted, deleted) - } - if origNum != num { - t.Fatalf("number of keys deleted did not match original execution: %d vs. %d", - origNum, num) - } - } - }) + // If at the same sequence as the initial DeleteRange. + if tc.sequence == origSeq { + if !reflect.DeepEqual(origDeleted, deleted) { + t.Fatalf("deleted keys did not match original execution: %+v vs. %+v", + origDeleted, deleted) + } + if origNum != num { + t.Fatalf("number of keys deleted did not match original execution: %d vs. %d", + origNum, num) + } } }) } @@ -3466,30 +3270,26 @@ func TestMVCCGetWithPushedTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - ctx := context.Background() - engine := engineImpl.create() - defer engine.Close() + ctx := context.Background() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Start with epoch 1. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { - t.Fatal(err) - } - // Resolve the intent, pushing its timestamp forward. - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1})); err != nil { - t.Fatal(err) - } - // Attempt to read using naive txn's previous timestamp. - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{ - Txn: txn1, - }) - if err != nil || value == nil || !bytes.Equal(value.RawBytes, value1.RawBytes) { - t.Errorf("expected value %q, err nil; got %+v, %v", value1.RawBytes, value, err) - } - }) + // Start with epoch 1. + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + t.Fatal(err) + } + // Resolve the intent, pushing its timestamp forward. + txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1})); err != nil { + t.Fatal(err) + } + // Attempt to read using naive txn's previous timestamp. + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{ + Txn: txn1, + }) + if err != nil || value == nil || !bytes.Equal(value.RawBytes, value1.RawBytes) { + t.Errorf("expected value %q, err nil; got %+v, %v", value1.RawBytes, value, err) } } @@ -3498,44 +3298,40 @@ func TestMVCCResolveWithDiffEpochs(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1e2); err != nil { - t.Fatal(err) - } - num, _, err := MVCCResolveWriteIntentRange(ctx, engine, nil, - roachpb.MakeLockUpdate(txn1e2Commit, roachpb.Span{Key: testKey1, EndKey: testKey2.Next()}), - 2) - if err != nil { - t.Fatal(err) - } - if num != 2 { - t.Errorf("expected 2 rows resolved; got %d", num) - } + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1e2); err != nil { + t.Fatal(err) + } + num, _, err := MVCCResolveWriteIntentRange(ctx, engine, nil, + roachpb.MakeLockUpdate(txn1e2Commit, roachpb.Span{Key: testKey1, EndKey: testKey2.Next()}), + 2) + if err != nil { + t.Fatal(err) + } + if num != 2 { + t.Errorf("expected 2 rows resolved; got %d", num) + } - // Verify key1 is empty, as resolution with epoch 2 would have - // aborted the epoch 1 intent. - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) - if value != nil || err != nil { - t.Errorf("expected value nil, err nil; got %+v, %v", value, err) - } + // Verify key1 is empty, as resolution with epoch 2 would have + // aborted the epoch 1 intent. + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + if value != nil || err != nil { + t.Errorf("expected value nil, err nil; got %+v, %v", value, err) + } - // Key2 should be committed. - value, _, err = MVCCGet(ctx, engine, testKey2, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value2.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value2.RawBytes, value.RawBytes) - } - }) + // Key2 should be committed. + value, _, err = MVCCGet(ctx, engine, testKey2, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value2.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value2.RawBytes, value.RawBytes) } } @@ -3544,51 +3340,47 @@ func TestMVCCResolveWithUpdatedTimestamp(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + t.Fatal(err) + } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ - Txn: txn1, - }) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + Txn: txn1, + }) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value1.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value1.RawBytes, value.RawBytes) + } - // Resolve with a higher commit timestamp -- this should rewrite the - // intent when making it permanent. - txn := makeTxn(*txn1Commit, hlc.Timestamp{WallTime: 1}) - if _, err = MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1})); err != nil { - t.Fatal(err) - } + // Resolve with a higher commit timestamp -- this should rewrite the + // intent when making it permanent. + txn := makeTxn(*txn1Commit, hlc.Timestamp{WallTime: 1}) + if _, err = MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1})); err != nil { + t.Fatal(err) + } - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) - if value != nil || err != nil { - t.Fatalf("expected both value and err to be nil: %+v, %v", value, err) - } + value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + if value != nil || err != nil { + t.Fatalf("expected both value and err to be nil: %+v, %v", value, err) + } - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) - if err != nil { - t.Error(err) - } - if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { - t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - }) + value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) + if err != nil { + t.Error(err) + } + if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { + t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) + } + if !bytes.Equal(value1.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value1.RawBytes, value.RawBytes) } } @@ -3597,53 +3389,49 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { - t.Fatal(err) - } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ - Txn: txn1, - }) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + t.Fatal(err) + } + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + Txn: txn1, + }) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value1.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value1.RawBytes, value.RawBytes) + } - // Resolve with a higher commit timestamp, but with still-pending transaction. - // This represents a straightforward push (i.e. from a read/write conflict). - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if _, err = MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1})); err != nil { - t.Fatal(err) - } + // Resolve with a higher commit timestamp, but with still-pending transaction. + // This represents a straightforward push (i.e. from a read/write conflict). + txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + if _, err = MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1})); err != nil { + t.Fatal(err) + } - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) - if value != nil || err == nil { - t.Fatalf("expected both value nil and err to be a writeIntentError: %+v", value) - } + value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) + if value != nil || err == nil { + t.Fatalf("expected both value nil and err to be a writeIntentError: %+v", value) + } - // Can still fetch the value using txn1. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ - Txn: txn1, - }) - if err != nil { - t.Error(err) - } - if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { - t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - }) + // Can still fetch the value using txn1. + value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + Txn: txn1, + }) + if err != nil { + t.Error(err) + } + if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { + t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) + } + if !bytes.Equal(value1.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value1.RawBytes, value.RawBytes) } } @@ -3652,38 +3440,34 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Resolve a non existent key; noop. - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn1Commit, roachpb.Span{Key: testKey1})); err != nil { - t.Fatal(err) - } + // Resolve a non existent key; noop. + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn1Commit, roachpb.Span{Key: testKey1})); err != nil { + t.Fatal(err) + } - // Add key and resolve despite there being no intent. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { - t.Fatal(err) - } - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn2Commit, roachpb.Span{Key: testKey1})); err != nil { - t.Fatal(err) - } + // Add key and resolve despite there being no intent. + if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value1, nil); err != nil { + t.Fatal(err) + } + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn2Commit, roachpb.Span{Key: testKey1})); err != nil { + t.Fatal(err) + } - // Write intent and resolve with different txn. - if err := MVCCPut(ctx, engine, nil, testKey2, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1); err != nil { - t.Fatal(err) - } + // Write intent and resolve with different txn. + if err := MVCCPut(ctx, engine, nil, testKey2, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn1); err != nil { + t.Fatal(err) + } - txn1CommitWithTS := txn2Commit.Clone() - txn1CommitWithTS.WriteTimestamp = hlc.Timestamp{WallTime: 1} - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn1CommitWithTS, roachpb.Span{Key: testKey2})); err != nil { - t.Fatal(err) - } - }) + txn1CommitWithTS := txn2Commit.Clone() + txn1CommitWithTS.WriteTimestamp = hlc.Timestamp{WallTime: 1} + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn1CommitWithTS, roachpb.Span{Key: testKey2})); err != nil { + t.Fatal(err) } } @@ -3692,78 +3476,74 @@ func TestMVCCResolveTxnRange(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey3, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, testKey4, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value4, txn1); err != nil { - t.Fatal(err) - } + if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, value2, nil); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey3, txn2.ReadTimestamp, hlc.ClockTimestamp{}, value3, txn2); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, testKey4, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value4, txn1); err != nil { + t.Fatal(err) + } - num, resumeSpan, err := MVCCResolveWriteIntentRange(ctx, engine, nil, - roachpb.MakeLockUpdate(txn1Commit, roachpb.Span{Key: testKey1, EndKey: testKey4.Next()}), - math.MaxInt64) - if err != nil { - t.Fatal(err) - } - if num != 2 || resumeSpan != nil { - t.Fatalf("expected all keys to process for resolution, even though 2 are noops; got %d, resume=%s", - num, resumeSpan) - } + num, resumeSpan, err := MVCCResolveWriteIntentRange(ctx, engine, nil, + roachpb.MakeLockUpdate(txn1Commit, roachpb.Span{Key: testKey1, EndKey: testKey4.Next()}), + math.MaxInt64) + if err != nil { + t.Fatal(err) + } + if num != 2 || resumeSpan != nil { + t.Fatalf("expected all keys to process for resolution, even though 2 are noops; got %d, resume=%s", + num, resumeSpan) + } - { - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value1.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - } - { - value, _, err := MVCCGet(ctx, engine, testKey2, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value2.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value2.RawBytes, value.RawBytes) - } - } - { - value, _, err := MVCCGet(ctx, engine, testKey3, hlc.Timestamp{Logical: 1}, MVCCGetOptions{ - Txn: txn2, - }) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value3.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value3.RawBytes, value.RawBytes) - } - } - { - value, _, err := MVCCGet(ctx, engine, testKey4, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) - if err != nil { - t.Fatal(err) - } - if !bytes.Equal(value4.RawBytes, value.RawBytes) { - t.Fatalf("the value %s in get result does not match the value %s in request", - value1.RawBytes, value.RawBytes) - } - } + { + value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value1.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value1.RawBytes, value.RawBytes) + } + } + { + value, _, err := MVCCGet(ctx, engine, testKey2, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value2.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value2.RawBytes, value.RawBytes) + } + } + { + value, _, err := MVCCGet(ctx, engine, testKey3, hlc.Timestamp{Logical: 1}, MVCCGetOptions{ + Txn: txn2, }) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value3.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value3.RawBytes, value.RawBytes) + } + } + { + value, _, err := MVCCGet(ctx, engine, testKey4, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + if err != nil { + t.Fatal(err) + } + if !bytes.Equal(value4.RawBytes, value.RawBytes) { + t.Fatalf("the value %s in get result does not match the value %s in request", + value1.RawBytes, value.RawBytes) + } } } @@ -3772,61 +3552,57 @@ func TestMVCCResolveTxnRangeResume(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Write 10 keys from txn1, 10 from txn2, and 10 with no txn, - // interleaved. The length of these keys changes and is non-decreasing. - // This exercises a subtle bug where separatedIntentAndVersionIter - // forgot to update its intentKey, but in some cases the shared slice - // for the unsafe key caused it to be inadvertently updated in a correct - // way. - for i := 0; i < 30; i += 3 { - key0 := roachpb.Key(fmt.Sprintf("%02d%d", i+0, i+0)) - key1 := roachpb.Key(fmt.Sprintf("%02d%d", i+1, i+1)) - key2 := roachpb.Key(fmt.Sprintf("%02d%d", i+2, i+2)) - if err := MVCCPut(ctx, engine, nil, key0, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { - t.Fatal(err) - } - txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 2}) - if err := MVCCPut(ctx, engine, nil, key1, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn2ts); err != nil { - t.Fatal(err) - } - if err := MVCCPut(ctx, engine, nil, key2, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { - t.Fatal(err) - } - } + // Write 10 keys from txn1, 10 from txn2, and 10 with no txn, + // interleaved. The length of these keys changes and is non-decreasing. + // This exercises a subtle bug where separatedIntentAndVersionIter + // forgot to update its intentKey, but in some cases the shared slice + // for the unsafe key caused it to be inadvertently updated in a correct + // way. + for i := 0; i < 30; i += 3 { + key0 := roachpb.Key(fmt.Sprintf("%02d%d", i+0, i+0)) + key1 := roachpb.Key(fmt.Sprintf("%02d%d", i+1, i+1)) + key2 := roachpb.Key(fmt.Sprintf("%02d%d", i+2, i+2)) + if err := MVCCPut(ctx, engine, nil, key0, txn1.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1); err != nil { + t.Fatal(err) + } + txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 2}) + if err := MVCCPut(ctx, engine, nil, key1, txn2ts.ReadTimestamp, hlc.ClockTimestamp{}, value2, txn2ts); err != nil { + t.Fatal(err) + } + if err := MVCCPut(ctx, engine, nil, key2, hlc.Timestamp{Logical: 3}, hlc.ClockTimestamp{}, value3, nil); err != nil { + t.Fatal(err) + } + } - rw := engine.NewBatch() - defer rw.Close() + rw := engine.NewBatch() + defer rw.Close() - // Resolve up to 6 intents: the keys are 000, 033, 066, 099, 1212, 1515. - num, resumeSpan, err := MVCCResolveWriteIntentRange(ctx, rw, nil, - roachpb.MakeLockUpdate(txn1Commit, roachpb.Span{Key: roachpb.Key("00"), EndKey: roachpb.Key("33")}), - 6) - if err != nil { - t.Fatal(err) - } - if num != 6 || resumeSpan == nil { - t.Errorf("expected resolution for only 6 keys; got %d, resume=%s", num, resumeSpan) - } - expResumeSpan := roachpb.Span{Key: roachpb.Key("1515").Next(), EndKey: roachpb.Key("33")} - if !resumeSpan.Equal(expResumeSpan) { - t.Errorf("expected resume span %s; got %s", expResumeSpan, resumeSpan) - } - require.NoError(t, rw.Commit(true)) - // Check that the intents are actually gone by trying to read above them - // using txn2. - for i := 0; i < 18; i += 3 { - val, intent, err := MVCCGet(ctx, engine, roachpb.Key(fmt.Sprintf("%02d%d", i, i)), - txn2.ReadTimestamp, MVCCGetOptions{Txn: txn2}) - require.NotNil(t, val) - require.NoError(t, err) - require.Nil(t, intent) - } - }) + // Resolve up to 6 intents: the keys are 000, 033, 066, 099, 1212, 1515. + num, resumeSpan, err := MVCCResolveWriteIntentRange(ctx, rw, nil, + roachpb.MakeLockUpdate(txn1Commit, roachpb.Span{Key: roachpb.Key("00"), EndKey: roachpb.Key("33")}), + 6) + if err != nil { + t.Fatal(err) + } + if num != 6 || resumeSpan == nil { + t.Errorf("expected resolution for only 6 keys; got %d, resume=%s", num, resumeSpan) + } + expResumeSpan := roachpb.Span{Key: roachpb.Key("1515").Next(), EndKey: roachpb.Key("33")} + if !resumeSpan.Equal(expResumeSpan) { + t.Errorf("expected resume span %s; got %s", expResumeSpan, resumeSpan) + } + require.NoError(t, rw.Commit(true)) + // Check that the intents are actually gone by trying to read above them + // using txn2. + for i := 0; i < 18; i += 3 { + val, intent, err := MVCCGet(ctx, engine, roachpb.Key(fmt.Sprintf("%02d%d", i, i)), + txn2.ReadTimestamp, MVCCGetOptions{Txn: txn2}) + require.NotNil(t, val) + require.NoError(t, err) + require.Nil(t, intent) } } @@ -3837,43 +3613,39 @@ func TestMVCCResolveTxnRangeResumeWithManyVersions(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Write 1000 keys with intents of which 100 are by the txn for which we - // will perform intent resolution. - lockUpdate := setupKeysWithIntent(t, engine, 5, /* numVersions */ - 1 /* numFlushedVersions */, false, 10, - false /* resolveIntentForLatestVersionWhenNonLockUpdateTxn */) - lockUpdate.Key = makeKey(nil, 0) - lockUpdate.EndKey = makeKey(nil, numIntentKeys) - i := 0 - for { - // Resolve up to 20 intents. - num, resumeSpan, err := MVCCResolveWriteIntentRange(ctx, engine, nil, lockUpdate, - 20) - require.NoError(t, err) - if resumeSpan == nil { - // Last call resolves 0 intents. - require.Equal(t, int64(0), num) - break - } - require.Equal(t, int64(20), num) - i++ - expResumeSpan := roachpb.Span{ - Key: makeKey(nil, (i*20-1)*10).Next(), - EndKey: lockUpdate.EndKey, - } - if !resumeSpan.Equal(expResumeSpan) { - t.Errorf("expected resume span %s; got %s", expResumeSpan, resumeSpan) - } - lockUpdate.Span = expResumeSpan - } - require.Equal(t, 5, i) - }) + // Write 1000 keys with intents of which 100 are by the txn for which we + // will perform intent resolution. + lockUpdate := setupKeysWithIntent(t, engine, 5, /* numVersions */ + 1 /* numFlushedVersions */, false, 10, + false /* resolveIntentForLatestVersionWhenNonLockUpdateTxn */) + lockUpdate.Key = makeKey(nil, 0) + lockUpdate.EndKey = makeKey(nil, numIntentKeys) + i := 0 + for { + // Resolve up to 20 intents. + num, resumeSpan, err := MVCCResolveWriteIntentRange(ctx, engine, nil, lockUpdate, + 20) + require.NoError(t, err) + if resumeSpan == nil { + // Last call resolves 0 intents. + require.Equal(t, int64(0), num) + break + } + require.Equal(t, int64(20), num) + i++ + expResumeSpan := roachpb.Span{ + Key: makeKey(nil, (i*20-1)*10).Next(), + EndKey: lockUpdate.EndKey, + } + if !resumeSpan.Equal(expResumeSpan) { + t.Errorf("expected resume span %s; got %s", expResumeSpan, resumeSpan) + } + lockUpdate.Span = expResumeSpan } + require.Equal(t, 5, i) } func generateBytes(rng *rand.Rand, min int, max int) []byte { @@ -4292,54 +4064,50 @@ func TestFindSplitKey(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - ms := &enginepb.MVCCStats{} - // Generate a series of KeyValues, each containing targetLength - // bytes, writing key #i to (encoded) key #i through the MVCC - // facility. Assuming that this translates roughly into same-length - // values after MVCC encoding, the split key should hence be chosen - // as the middle key of the interval. - splitReservoirSize := 100 - for i := 0; i < splitReservoirSize; i++ { - k := fmt.Sprintf("%09d", i) - v := strings.Repeat("X", 10-len(k)) - val := roachpb.MakeValueFromString(v) - // Write the key and value through MVCC - if err := MVCCPut(ctx, engine, ms, []byte(k), hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, val, nil); err != nil { - t.Fatal(err) - } - } + ms := &enginepb.MVCCStats{} + // Generate a series of KeyValues, each containing targetLength + // bytes, writing key #i to (encoded) key #i through the MVCC + // facility. Assuming that this translates roughly into same-length + // values after MVCC encoding, the split key should hence be chosen + // as the middle key of the interval. + splitReservoirSize := 100 + for i := 0; i < splitReservoirSize; i++ { + k := fmt.Sprintf("%09d", i) + v := strings.Repeat("X", 10-len(k)) + val := roachpb.MakeValueFromString(v) + // Write the key and value through MVCC + if err := MVCCPut(ctx, engine, ms, []byte(k), hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, val, nil); err != nil { + t.Fatal(err) + } + } - testData := []struct { - targetSize int64 - splitInd int - }{ - {(ms.KeyBytes + ms.ValBytes) / 2, splitReservoirSize / 2}, - {0, 0}, - {math.MaxInt64, splitReservoirSize}, - } + testData := []struct { + targetSize int64 + splitInd int + }{ + {(ms.KeyBytes + ms.ValBytes) / 2, splitReservoirSize / 2}, + {0, 0}, + {math.MaxInt64, splitReservoirSize}, + } - for i, td := range testData { - humanSplitKey, err := MVCCFindSplitKey(ctx, engine, roachpb.RKeyMin, roachpb.RKeyMax, td.targetSize) - if err != nil { - t.Fatal(err) - } - ind, err := strconv.Atoi(string(humanSplitKey)) - if err != nil { - t.Fatalf("%d: could not parse key %s as int: %+v", i, humanSplitKey, err) - } - if ind == 0 { - t.Fatalf("%d: should never select first key as split key", i) - } - if diff := td.splitInd - ind; diff > 1 || diff < -1 { - t.Fatalf("%d: wanted key #%d+-1, but got %d (diff %d)", i, td.splitInd, ind, diff) - } - } - }) + for i, td := range testData { + humanSplitKey, err := MVCCFindSplitKey(ctx, engine, roachpb.RKeyMin, roachpb.RKeyMax, td.targetSize) + if err != nil { + t.Fatal(err) + } + ind, err := strconv.Atoi(string(humanSplitKey)) + if err != nil { + t.Fatalf("%d: could not parse key %s as int: %+v", i, humanSplitKey, err) + } + if ind == 0 { + t.Fatalf("%d: should never select first key as split key", i) + } + if diff := td.splitInd - ind; diff > 1 || diff < -1 { + t.Fatalf("%d: wanted key #%d+-1, but got %d (diff %d)", i, td.splitInd, ind, diff) + } } } @@ -4581,68 +4349,64 @@ func TestFindValidSplitKeys(t *testing.T) { }, } - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - testutils.RunTrueAndFalse(t, "tenant", func(t *testing.T, tenant bool) { - for i, test := range testCases { - t.Run("", func(t *testing.T) { - if tenant { - if test.skipTenant { - skip.IgnoreLint(t, "") - } - // Update all keys to include a tenant prefix. - tenPrefix := keys.MakeSQLCodec(roachpb.MinTenantID).TenantPrefix() - test = prefixTestKeys(test, tenPrefix) - } + testutils.RunTrueAndFalse(t, "tenant", func(t *testing.T, tenant bool) { + for i, test := range testCases { + t.Run("", func(t *testing.T) { + if tenant { + if test.skipTenant { + skip.IgnoreLint(t, "") + } + // Update all keys to include a tenant prefix. + tenPrefix := keys.MakeSQLCodec(roachpb.MinTenantID).TenantPrefix() + test = prefixTestKeys(test, tenPrefix) + } - ctx := context.Background() - engine := engineImpl.create() - defer engine.Close() - - ms := &enginepb.MVCCStats{} - val := roachpb.MakeValueFromString(strings.Repeat("X", 10)) - for _, k := range test.keys { - // Add three MVCC versions of every key. Splits are not allowed - // between MVCC versions, so this shouldn't have any effect. - for j := 1; j <= 3; j++ { - ts := hlc.Timestamp{Logical: int32(j)} - if err := MVCCPut(ctx, engine, ms, []byte(k), ts, hlc.ClockTimestamp{}, val, nil); err != nil { - t.Fatal(err) - } - } - } - rangeStart := test.keys[0] - if len(test.rangeStart) > 0 { - rangeStart = test.rangeStart - } - rangeEnd := test.keys[len(test.keys)-1].Next() - rangeStartAddr, err := keys.Addr(rangeStart) - if err != nil { - t.Fatal(err) - } - rangeEndAddr, err := keys.Addr(rangeEnd) - if err != nil { + ctx := context.Background() + engine := NewDefaultInMemForTesting() + defer engine.Close() + + ms := &enginepb.MVCCStats{} + val := roachpb.MakeValueFromString(strings.Repeat("X", 10)) + for _, k := range test.keys { + // Add three MVCC versions of every key. Splits are not allowed + // between MVCC versions, so this shouldn't have any effect. + for j := 1; j <= 3; j++ { + ts := hlc.Timestamp{Logical: int32(j)} + if err := MVCCPut(ctx, engine, ms, []byte(k), ts, hlc.ClockTimestamp{}, val, nil); err != nil { t.Fatal(err) } - targetSize := (ms.KeyBytes + ms.ValBytes) / 2 - splitKey, err := MVCCFindSplitKey(ctx, engine, rangeStartAddr, rangeEndAddr, targetSize) - if test.expError { - if !testutils.IsError(err, "has no valid splits") { - t.Fatalf("%d: unexpected error: %+v", i, err) - } - return - } - if err != nil { - t.Fatalf("%d; unexpected error: %+v", i, err) - } - if !splitKey.Equal(test.expSplit) { - t.Errorf("%d: expected split key %q; got %q", i, test.expSplit, splitKey) - } - }) + } + } + rangeStart := test.keys[0] + if len(test.rangeStart) > 0 { + rangeStart = test.rangeStart + } + rangeEnd := test.keys[len(test.keys)-1].Next() + rangeStartAddr, err := keys.Addr(rangeStart) + if err != nil { + t.Fatal(err) + } + rangeEndAddr, err := keys.Addr(rangeEnd) + if err != nil { + t.Fatal(err) + } + targetSize := (ms.KeyBytes + ms.ValBytes) / 2 + splitKey, err := MVCCFindSplitKey(ctx, engine, rangeStartAddr, rangeEndAddr, targetSize) + if test.expError { + if !testutils.IsError(err, "has no valid splits") { + t.Fatalf("%d: unexpected error: %+v", i, err) + } + return + } + if err != nil { + t.Fatalf("%d; unexpected error: %+v", i, err) + } + if !splitKey.Equal(test.expSplit) { + t.Errorf("%d: expected split key %q; got %q", i, test.expSplit, splitKey) } }) - }) - } + } + }) } // TestFindBalancedSplitKeys verifies split keys are located such that @@ -4693,35 +4457,31 @@ func TestFindBalancedSplitKeys(t *testing.T) { }, } - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - for i, test := range testCases { - t.Run("", func(t *testing.T) { - ctx := context.Background() - engine := engineImpl.create() - defer engine.Close() - - ms := &enginepb.MVCCStats{} - var expKey roachpb.Key - for j, keySize := range test.keySizes { - key := roachpb.Key(fmt.Sprintf("%d%s", j, strings.Repeat("X", keySize))) - if test.expSplit == j { - expKey = key - } - val := roachpb.MakeValueFromString(strings.Repeat("X", test.valSizes[j])) - if err := MVCCPut(ctx, engine, ms, key, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, val, nil); err != nil { - t.Fatal(err) - } - } - targetSize := (ms.KeyBytes + ms.ValBytes) / 2 - splitKey, err := MVCCFindSplitKey(ctx, engine, roachpb.RKey("\x02"), roachpb.RKeyMax, targetSize) - if err != nil { - t.Fatalf("unexpected error: %+v", err) - } - if !splitKey.Equal(expKey) { - t.Errorf("%d: expected split key %q; got %q", i, expKey, splitKey) - } - }) + for i, test := range testCases { + t.Run("", func(t *testing.T) { + ctx := context.Background() + engine := NewDefaultInMemForTesting() + defer engine.Close() + + ms := &enginepb.MVCCStats{} + var expKey roachpb.Key + for j, keySize := range test.keySizes { + key := roachpb.Key(fmt.Sprintf("%d%s", j, strings.Repeat("X", keySize))) + if test.expSplit == j { + expKey = key + } + val := roachpb.MakeValueFromString(strings.Repeat("X", test.valSizes[j])) + if err := MVCCPut(ctx, engine, ms, key, hlc.Timestamp{Logical: 1}, hlc.ClockTimestamp{}, val, nil); err != nil { + t.Fatal(err) + } + } + targetSize := (ms.KeyBytes + ms.ValBytes) / 2 + splitKey, err := MVCCFindSplitKey(ctx, engine, roachpb.RKey("\x02"), roachpb.RKeyMax, targetSize) + if err != nil { + t.Fatalf("unexpected error: %+v", err) + } + if !splitKey.Equal(expKey) { + t.Errorf("%d: expected split key %q; got %q", i, expKey, splitKey) } }) } @@ -4735,162 +4495,158 @@ func TestMVCCGarbageCollect(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - ms := &enginepb.MVCCStats{} + ms := &enginepb.MVCCStats{} - val := []byte("value") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} - ts3 := hlc.Timestamp{WallTime: 3e9} - ts4 := hlc.Timestamp{WallTime: 4e9} - ts5 := hlc.Timestamp{WallTime: 4e9} - val1 := roachpb.MakeValueFromBytesAndTimestamp(val, ts1) - val2 := roachpb.MakeValueFromBytesAndTimestamp(val, ts2) - val3 := roachpb.MakeValueFromBytesAndTimestamp(val, ts3) - valInline := roachpb.MakeValueFromBytesAndTimestamp(val, hlc.Timestamp{}) - - testData := []struct { - key roachpb.Key - vals []roachpb.Value - isDeleted bool // is the most recent value a deletion tombstone? - }{ - {roachpb.Key("a"), []roachpb.Value{val1, val2}, false}, - {roachpb.Key("a-del"), []roachpb.Value{val1, val2}, true}, - {roachpb.Key("b"), []roachpb.Value{val1, val2, val3}, false}, - {roachpb.Key("b-del"), []roachpb.Value{val1, val2, val3}, true}, - {roachpb.Key("inline"), []roachpb.Value{valInline}, false}, - {roachpb.Key("r-2"), []roachpb.Value{val1}, false}, - {roachpb.Key("r-3"), []roachpb.Value{val1}, false}, - {roachpb.Key("r-4"), []roachpb.Value{val1}, false}, - {roachpb.Key("r-6"), []roachpb.Value{val1}, true}, - {roachpb.Key("t"), []roachpb.Value{val1}, false}, - } + val := []byte("value") + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2e9} + ts3 := hlc.Timestamp{WallTime: 3e9} + ts4 := hlc.Timestamp{WallTime: 4e9} + ts5 := hlc.Timestamp{WallTime: 4e9} + val1 := roachpb.MakeValueFromBytesAndTimestamp(val, ts1) + val2 := roachpb.MakeValueFromBytesAndTimestamp(val, ts2) + val3 := roachpb.MakeValueFromBytesAndTimestamp(val, ts3) + valInline := roachpb.MakeValueFromBytesAndTimestamp(val, hlc.Timestamp{}) - for i := 0; i < 3; i++ { - for _, test := range testData { - if i >= len(test.vals) { - continue - } - for _, val := range test.vals[i : i+1] { - if i == len(test.vals)-1 && test.isDeleted { - if _, err := MVCCDelete(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, - nil); err != nil { - t.Fatal(err) - } - continue - } - valCpy := *protoutil.Clone(&val).(*roachpb.Value) - valCpy.Timestamp = hlc.Timestamp{} - if err := MVCCPut(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, - valCpy, nil); err != nil { - t.Fatal(err) - } + testData := []struct { + key roachpb.Key + vals []roachpb.Value + isDeleted bool // is the most recent value a deletion tombstone? + }{ + {roachpb.Key("a"), []roachpb.Value{val1, val2}, false}, + {roachpb.Key("a-del"), []roachpb.Value{val1, val2}, true}, + {roachpb.Key("b"), []roachpb.Value{val1, val2, val3}, false}, + {roachpb.Key("b-del"), []roachpb.Value{val1, val2, val3}, true}, + {roachpb.Key("inline"), []roachpb.Value{valInline}, false}, + {roachpb.Key("r-2"), []roachpb.Value{val1}, false}, + {roachpb.Key("r-3"), []roachpb.Value{val1}, false}, + {roachpb.Key("r-4"), []roachpb.Value{val1}, false}, + {roachpb.Key("r-6"), []roachpb.Value{val1}, true}, + {roachpb.Key("t"), []roachpb.Value{val1}, false}, + } + + for i := 0; i < 3; i++ { + for _, test := range testData { + if i >= len(test.vals) { + continue + } + for _, val := range test.vals[i : i+1] { + if i == len(test.vals)-1 && test.isDeleted { + if _, err := MVCCDelete(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, + nil); err != nil { + t.Fatal(err) } + continue } - } - if err := MVCCDeleteRangeUsingTombstone(ctx, engine, ms, roachpb.Key("r"), - roachpb.Key("r-del").Next(), ts3, hlc.ClockTimestamp{}, nil, nil, false, 0, nil); err != nil { - t.Fatal(err) - } - if err := MVCCDeleteRangeUsingTombstone(ctx, engine, ms, roachpb.Key("t"), - roachpb.Key("u").Next(), ts2, hlc.ClockTimestamp{}, nil, nil, false, 0, nil); err != nil { - t.Fatal(err) - } - if err := MVCCDeleteRangeUsingTombstone(ctx, engine, ms, roachpb.Key("t"), - roachpb.Key("u").Next(), ts3, hlc.ClockTimestamp{}, nil, nil, false, 0, nil); err != nil { - t.Fatal(err) - } - if log.V(1) { - log.Info(context.Background(), "Engine content before GC") - kvsn, err := Scan(engine, localMax, keyMax, 0) - if err != nil { + valCpy := *protoutil.Clone(&val).(*roachpb.Value) + valCpy.Timestamp = hlc.Timestamp{} + if err := MVCCPut(ctx, engine, ms, test.key, val.Timestamp, hlc.ClockTimestamp{}, + valCpy, nil); err != nil { t.Fatal(err) } - for i, kv := range kvsn { - log.Infof(context.Background(), "%d: %s", i, kv.Key) - } } + } + } + if err := MVCCDeleteRangeUsingTombstone(ctx, engine, ms, roachpb.Key("r"), + roachpb.Key("r-del").Next(), ts3, hlc.ClockTimestamp{}, nil, nil, false, 0, nil); err != nil { + t.Fatal(err) + } + if err := MVCCDeleteRangeUsingTombstone(ctx, engine, ms, roachpb.Key("t"), + roachpb.Key("u").Next(), ts2, hlc.ClockTimestamp{}, nil, nil, false, 0, nil); err != nil { + t.Fatal(err) + } + if err := MVCCDeleteRangeUsingTombstone(ctx, engine, ms, roachpb.Key("t"), + roachpb.Key("u").Next(), ts3, hlc.ClockTimestamp{}, nil, nil, false, 0, nil); err != nil { + t.Fatal(err) + } + if log.V(1) { + log.Info(context.Background(), "Engine content before GC") + kvsn, err := Scan(engine, localMax, keyMax, 0) + if err != nil { + t.Fatal(err) + } + for i, kv := range kvsn { + log.Infof(context.Background(), "%d: %s", i, kv.Key) + } + } - gcTime := ts5 - gcKeys := []roachpb.GCRequest_GCKey{ - {Key: roachpb.Key("a"), Timestamp: ts1}, - {Key: roachpb.Key("a-del"), Timestamp: ts2}, - {Key: roachpb.Key("b"), Timestamp: ts1}, - {Key: roachpb.Key("b-del"), Timestamp: ts2}, - {Key: roachpb.Key("inline"), Timestamp: hlc.Timestamp{}}, - // Keys that don't exist, which should result in a no-op. - {Key: roachpb.Key("a-bad"), Timestamp: ts2}, - {Key: roachpb.Key("inline-bad"), Timestamp: hlc.Timestamp{}}, - // Keys that are hidden by range key. - // Non-existing keys that needs to skip gracefully without - // distorting stats. (Checking that following keys doesn't affect it) - {Key: roachpb.Key("r-0"), Timestamp: ts1}, - {Key: roachpb.Key("r-1"), Timestamp: ts4}, - // Request has a timestamp below range key, it will be handled by - // logic processing range tombstones specifically. - {Key: roachpb.Key("r-2"), Timestamp: ts1}, - // Requests has a timestamp at or above range key, it will be handled by - // logic processing synthesized metadata. - {Key: roachpb.Key("r-3"), Timestamp: ts3}, - {Key: roachpb.Key("r-4"), Timestamp: ts4}, - // This is a non-existing key that needs to skip gracefully without - // distorting stats. Checking that absence of next key is handled. - {Key: roachpb.Key("r-5"), Timestamp: ts4}, - // Delete key covered by range delete key. - {Key: roachpb.Key("r-6"), Timestamp: ts4}, - - {Key: roachpb.Key("t"), Timestamp: ts4}, - } - if err := MVCCGarbageCollect( - context.Background(), engine, ms, gcKeys, gcTime, - ); err != nil { - t.Fatal(err) - } + gcTime := ts5 + gcKeys := []roachpb.GCRequest_GCKey{ + {Key: roachpb.Key("a"), Timestamp: ts1}, + {Key: roachpb.Key("a-del"), Timestamp: ts2}, + {Key: roachpb.Key("b"), Timestamp: ts1}, + {Key: roachpb.Key("b-del"), Timestamp: ts2}, + {Key: roachpb.Key("inline"), Timestamp: hlc.Timestamp{}}, + // Keys that don't exist, which should result in a no-op. + {Key: roachpb.Key("a-bad"), Timestamp: ts2}, + {Key: roachpb.Key("inline-bad"), Timestamp: hlc.Timestamp{}}, + // Keys that are hidden by range key. + // Non-existing keys that needs to skip gracefully without + // distorting stats. (Checking that following keys doesn't affect it) + {Key: roachpb.Key("r-0"), Timestamp: ts1}, + {Key: roachpb.Key("r-1"), Timestamp: ts4}, + // Request has a timestamp below range key, it will be handled by + // logic processing range tombstones specifically. + {Key: roachpb.Key("r-2"), Timestamp: ts1}, + // Requests has a timestamp at or above range key, it will be handled by + // logic processing synthesized metadata. + {Key: roachpb.Key("r-3"), Timestamp: ts3}, + {Key: roachpb.Key("r-4"), Timestamp: ts4}, + // This is a non-existing key that needs to skip gracefully without + // distorting stats. Checking that absence of next key is handled. + {Key: roachpb.Key("r-5"), Timestamp: ts4}, + // Delete key covered by range delete key. + {Key: roachpb.Key("r-6"), Timestamp: ts4}, + + {Key: roachpb.Key("t"), Timestamp: ts4}, + } + if err := MVCCGarbageCollect( + context.Background(), engine, ms, gcKeys, gcTime, + ); err != nil { + t.Fatal(err) + } - if log.V(1) { - log.Info(context.Background(), "Engine content after GC") - kvsn, err := Scan(engine, localMax, keyMax, 0) - if err != nil { - t.Fatal(err) - } - for i, kv := range kvsn { - log.Infof(context.Background(), "%d: %s", i, kv.Key) - } - } + if log.V(1) { + log.Info(context.Background(), "Engine content after GC") + kvsn, err := Scan(engine, localMax, keyMax, 0) + if err != nil { + t.Fatal(err) + } + for i, kv := range kvsn { + log.Infof(context.Background(), "%d: %s", i, kv.Key) + } + } - expEncKeys := []MVCCKey{ - mvccVersionKey(roachpb.Key("a"), ts2), - mvccVersionKey(roachpb.Key("b"), ts3), - mvccVersionKey(roachpb.Key("b"), ts2), - mvccVersionKey(roachpb.Key("b-del"), ts3), - } - kvs, err := Scan(engine, localMax, keyMax, 0) + expEncKeys := []MVCCKey{ + mvccVersionKey(roachpb.Key("a"), ts2), + mvccVersionKey(roachpb.Key("b"), ts3), + mvccVersionKey(roachpb.Key("b"), ts2), + mvccVersionKey(roachpb.Key("b-del"), ts3), + } + kvs, err := Scan(engine, localMax, keyMax, 0) + if err != nil { + t.Fatal(err) + } + if len(kvs) != len(expEncKeys) { + t.Fatalf("number of kvs %d != expected %d", len(kvs), len(expEncKeys)) + } + for i, kv := range kvs { + if !kv.Key.Equal(expEncKeys[i]) { + t.Errorf("%d: expected key %q; got %q", i, expEncKeys[i], kv.Key) + } + } + + // Verify aggregated stats match computed stats after GC. + for _, mvccStatsTest := range mvccStatsTests { + t.Run(mvccStatsTest.name, func(t *testing.T) { + expMS, err := mvccStatsTest.fn(engine, localMax, roachpb.KeyMax, gcTime.WallTime) if err != nil { t.Fatal(err) } - if len(kvs) != len(expEncKeys) { - t.Fatalf("number of kvs %d != expected %d", len(kvs), len(expEncKeys)) - } - for i, kv := range kvs { - if !kv.Key.Equal(expEncKeys[i]) { - t.Errorf("%d: expected key %q; got %q", i, expEncKeys[i], kv.Key) - } - } - - // Verify aggregated stats match computed stats after GC. - for _, mvccStatsTest := range mvccStatsTests { - t.Run(mvccStatsTest.name, func(t *testing.T) { - expMS, err := mvccStatsTest.fn(engine, localMax, roachpb.KeyMax, gcTime.WallTime) - if err != nil { - t.Fatal(err) - } - assertEq(t, engine, "verification", ms, &expMS) - }) - } + assertEq(t, engine, "verification", ms, &expMS) }) } } @@ -4902,45 +4658,41 @@ func TestMVCCGarbageCollectNonDeleted(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - s := "string" - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} - val1 := mkVal(s, ts1) - val2 := mkVal(s, ts2) - valInline := mkVal(s, hlc.Timestamp{}) - - testData := []struct { - key roachpb.Key - vals []roachpb.Value - expError string - }{ - {roachpb.Key("a"), []roachpb.Value{val1, val2}, `request to GC non-deleted, latest value of "a"`}, - {roachpb.Key("inline"), []roachpb.Value{valInline}, ""}, - } + s := "string" + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2e9} + val1 := mkVal(s, ts1) + val2 := mkVal(s, ts2) + valInline := mkVal(s, hlc.Timestamp{}) - for _, test := range testData { - for _, val := range test.vals { - valCpy := *protoutil.Clone(&val).(*roachpb.Value) - valCpy.Timestamp = hlc.Timestamp{} - if err := MVCCPut(ctx, engine, nil, test.key, val.Timestamp, hlc.ClockTimestamp{}, valCpy, nil); err != nil { - t.Fatal(err) - } - } - keys := []roachpb.GCRequest_GCKey{ - {Key: test.key, Timestamp: ts2}, - } - err := MVCCGarbageCollect(ctx, engine, nil, keys, ts2) - if !testutils.IsError(err, test.expError) { - t.Fatalf("expected error %q when garbage collecting a non-deleted live value, found %v", - test.expError, err) - } + testData := []struct { + key roachpb.Key + vals []roachpb.Value + expError string + }{ + {roachpb.Key("a"), []roachpb.Value{val1, val2}, `request to GC non-deleted, latest value of "a"`}, + {roachpb.Key("inline"), []roachpb.Value{valInline}, ""}, + } + + for _, test := range testData { + for _, val := range test.vals { + valCpy := *protoutil.Clone(&val).(*roachpb.Value) + valCpy.Timestamp = hlc.Timestamp{} + if err := MVCCPut(ctx, engine, nil, test.key, val.Timestamp, hlc.ClockTimestamp{}, valCpy, nil); err != nil { + t.Fatal(err) } - }) + } + keys := []roachpb.GCRequest_GCKey{ + {Key: test.key, Timestamp: ts2}, + } + err := MVCCGarbageCollect(ctx, engine, nil, keys, ts2) + if !testutils.IsError(err, test.expError) { + t.Fatalf("expected error %q when garbage collecting a non-deleted live value, found %v", + test.expError, err) + } } } @@ -4950,35 +4702,31 @@ func TestMVCCGarbageCollectIntent(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - bytes := []byte("value") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} - key := roachpb.Key("a") - { - val1 := roachpb.MakeValueFromBytes(bytes) - if err := MVCCPut(ctx, engine, nil, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { - t.Fatal(err) - } - } - txn := &roachpb.Transaction{ - TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, - ReadTimestamp: ts2, - } - if _, err := MVCCDelete(ctx, engine, nil, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { - t.Fatal(err) - } - keys := []roachpb.GCRequest_GCKey{ - {Key: key, Timestamp: ts2}, - } - if err := MVCCGarbageCollect(ctx, engine, nil, keys, ts2); err == nil { - t.Fatal("expected error garbage collecting an intent") - } - }) + bytes := []byte("value") + ts1 := hlc.Timestamp{WallTime: 1e9} + ts2 := hlc.Timestamp{WallTime: 2e9} + key := roachpb.Key("a") + { + val1 := roachpb.MakeValueFromBytes(bytes) + if err := MVCCPut(ctx, engine, nil, key, ts1, hlc.ClockTimestamp{}, val1, nil); err != nil { + t.Fatal(err) + } + } + txn := &roachpb.Transaction{ + TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts2}, + ReadTimestamp: ts2, + } + if _, err := MVCCDelete(ctx, engine, nil, key, txn.ReadTimestamp, hlc.ClockTimestamp{}, txn); err != nil { + t.Fatal(err) + } + keys := []roachpb.GCRequest_GCKey{ + {Key: key, Timestamp: ts2}, + } + if err := MVCCGarbageCollect(ctx, engine, nil, keys, ts2); err == nil { + t.Fatal("expected error garbage collecting an intent") } } @@ -4990,24 +4738,20 @@ func TestMVCCGarbageCollectPanicsWithMixOfLocalAndGlobalKeys(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - require.Panics(t, func() { - ts := hlc.Timestamp{WallTime: 1e9} - k := roachpb.Key("a") - keys := []roachpb.GCRequest_GCKey{ - {Key: k, Timestamp: ts}, - {Key: keys.RangeDescriptorKey(roachpb.RKey(k))}, - } - if err := MVCCGarbageCollect(ctx, engine, nil, keys, ts); err != nil { - panic(err) - } - }) - }) - } + require.Panics(t, func() { + ts := hlc.Timestamp{WallTime: 1e9} + k := roachpb.Key("a") + keys := []roachpb.GCRequest_GCKey{ + {Key: k, Timestamp: ts}, + {Key: keys.RangeDescriptorKey(roachpb.RKey(k))}, + } + if err := MVCCGarbageCollect(ctx, engine, nil, keys, ts); err != nil { + panic(err) + } + }) } // readWriterReturningSeekLTTrackingIterator is used in a test to inject errors @@ -5193,15 +4937,11 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { }, }, } - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - for _, tc := range cases { - t.Run(tc.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - runTestCase(t, tc, engine) - }) - } + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + engine := NewDefaultInMemForTesting() + defer engine.Close() + runTestCase(t, tc, engine) }) } } @@ -5700,57 +5440,53 @@ func TestMVCCGarbageCollectRanges(t *testing.T) { }, } - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - for _, d := range testData { - t.Run(d.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + for _, d := range testData { + t.Run(d.name, func(t *testing.T) { + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Populate range descriptor defaults. - if len(d.rangeStart) == 0 { - d.rangeStart = rangeStart - } - if len(d.rangeEnd) == 0 { - d.rangeEnd = rangeEnd - } + // Populate range descriptor defaults. + if len(d.rangeStart) == 0 { + d.rangeStart = rangeStart + } + if len(d.rangeEnd) == 0 { + d.rangeEnd = rangeEnd + } - var ms enginepb.MVCCStats - d.before.populateEngine(t, engine, &ms) - - rangeKeys := rangesFromRequests(rangeStart, rangeEnd, d.request) - require.NoError(t, MVCCGarbageCollectRangeKeys(ctx, engine, &ms, rangeKeys), - "failed to run mvcc range tombstone garbage collect") - - it := engine.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ - KeyTypes: IterKeyTypeRangesOnly, - LowerBound: d.rangeStart, - UpperBound: d.rangeEnd, - }) - defer it.Close() - it.SeekGE(MVCCKey{Key: d.rangeStart}) - expectIndex := 0 - for ; ; it.Next() { - ok, err := it.Valid() - require.NoError(t, err, "failed to iterate engine") - if !ok { - break - } - for _, rk := range it.RangeKeys().AsRangeKeys() { - require.Less(t, expectIndex, len(d.after), "not enough expectations; at unexpected range: %s", rk) - require.EqualValues(t, d.after[expectIndex], rk, "range key is not equal") - expectIndex++ - } - } - require.Equal(t, len(d.after), expectIndex, - "not all range tombstone expectations were consumed") - - ms.AgeTo(tsMax.WallTime) - expMs, err := ComputeStats(engine, d.rangeStart, d.rangeEnd, tsMax.WallTime) - require.NoError(t, err, "failed to compute stats for range") - require.EqualValues(t, expMs, ms, "computed range stats vs gc'd") - }) + var ms enginepb.MVCCStats + d.before.populateEngine(t, engine, &ms) + + rangeKeys := rangesFromRequests(rangeStart, rangeEnd, d.request) + require.NoError(t, MVCCGarbageCollectRangeKeys(ctx, engine, &ms, rangeKeys), + "failed to run mvcc range tombstone garbage collect") + + it := engine.NewMVCCIterator(MVCCKeyIterKind, IterOptions{ + KeyTypes: IterKeyTypeRangesOnly, + LowerBound: d.rangeStart, + UpperBound: d.rangeEnd, + }) + defer it.Close() + it.SeekGE(MVCCKey{Key: d.rangeStart}) + expectIndex := 0 + for ; ; it.Next() { + ok, err := it.Valid() + require.NoError(t, err, "failed to iterate engine") + if !ok { + break + } + for _, rk := range it.RangeKeys().AsRangeKeys() { + require.Less(t, expectIndex, len(d.after), "not enough expectations; at unexpected range: %s", rk) + require.EqualValues(t, d.after[expectIndex], rk, "range key is not equal") + expectIndex++ + } } + require.Equal(t, len(d.after), expectIndex, + "not all range tombstone expectations were consumed") + + ms.AgeTo(tsMax.WallTime) + expMs, err := ComputeStats(engine, d.rangeStart, d.rangeEnd, tsMax.WallTime) + require.NoError(t, err, "failed to compute stats for range") + require.EqualValues(t, expMs, ms, "computed range stats vs gc'd") }) } } @@ -5872,20 +5608,16 @@ func TestMVCCGarbageCollectRangesFailures(t *testing.T) { } ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - for _, d := range testData { - t.Run(d.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - d.before.populateEngine(t, engine, nil) - rangeKeys := rangesFromRequests(rangeStart, rangeEnd, d.request) - err := MVCCGarbageCollectRangeKeys(ctx, engine, nil, rangeKeys) - require.Errorf(t, err, "expected error '%s' but found none", d.error) - require.True(t, testutils.IsError(err, d.error), - "expected error '%s' found '%s'", d.error, err) - }) - } + for _, d := range testData { + t.Run(d.name, func(t *testing.T) { + engine := NewDefaultInMemForTesting() + defer engine.Close() + d.before.populateEngine(t, engine, nil) + rangeKeys := rangesFromRequests(rangeStart, rangeEnd, d.request) + err := MVCCGarbageCollectRangeKeys(ctx, engine, nil, rangeKeys) + require.Errorf(t, err, "expected error '%s' but found none", d.error) + require.True(t, testutils.IsError(err, d.error), + "expected error '%s' found '%s'", d.error, err) }) } } @@ -5933,36 +5665,32 @@ func TestMVCCGarbageCollectClearRange(t *testing.T) { } request := mkGCReq(keyA, keyD) - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - var ms, diff enginepb.MVCCStats - before.populateEngine(t, engine, &ms) + var ms, diff enginepb.MVCCStats + before.populateEngine(t, engine, &ms) - require.NoError(t, - MVCCGarbageCollectWholeRange(ctx, engine, &diff, request.StartKey, request.EndKey, tsGC, ms), - "failed to run mvcc range tombstone garbage collect") - ms.Add(diff) + require.NoError(t, + MVCCGarbageCollectWholeRange(ctx, engine, &diff, request.StartKey, request.EndKey, tsGC, ms), + "failed to run mvcc range tombstone garbage collect") + ms.Add(diff) - rks := scanRangeKeys(t, engine) - require.Empty(t, rks) - ks := scanPointKeys(t, engine) - require.Empty(t, ks) + rks := scanRangeKeys(t, engine) + require.Empty(t, rks) + ks := scanPointKeys(t, engine) + require.Empty(t, ks) - ms.AgeTo(tsMax.WallTime) - it := engine.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ - KeyTypes: IterKeyTypePointsAndRanges, - LowerBound: rangeStart, - UpperBound: rangeEnd, - }) - defer it.Close() - expMs, err := ComputeStatsForIter(it, tsMax.WallTime) - require.NoError(t, err, "failed to compute stats for range") - require.EqualValues(t, expMs, ms, "computed range stats vs gc'd") - }) - } + ms.AgeTo(tsMax.WallTime) + it := engine.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + LowerBound: rangeStart, + UpperBound: rangeEnd, + }) + defer it.Close() + expMs, err := ComputeStatsForIter(it, tsMax.WallTime) + require.NoError(t, err, "failed to compute stats for range") + require.EqualValues(t, expMs, ms, "computed range stats vs gc'd") } func TestMVCCGarbageCollectClearRangeInlinedValue(t *testing.T) { @@ -5999,24 +5727,20 @@ func TestMVCCGarbageCollectClearRangeInlinedValue(t *testing.T) { } request := mkGCReq(keyA, keyD) expectedError := `found key not covered by range tombstone /Table/42/"b"/0,0` - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - var ms, diff enginepb.MVCCStats - before.populateEngine(t, engine, &ms) - // We are forcing stats to be estimates to bypass quick liveness check - // that will prevent actual data checks if there's some live data. - ms.ContainsEstimates = 1 - err := MVCCGarbageCollectWholeRange(ctx, engine, &diff, request.StartKey, request.EndKey, - tsGC, ms) - ms.Add(diff) - require.Errorf(t, err, "expected error '%s' but found none", expectedError) - require.True(t, testutils.IsError(err, expectedError), - "expected error '%s' found '%s'", expectedError, err) - }) - } + var ms, diff enginepb.MVCCStats + before.populateEngine(t, engine, &ms) + // We are forcing stats to be estimates to bypass quick liveness check + // that will prevent actual data checks if there's some live data. + ms.ContainsEstimates = 1 + err := MVCCGarbageCollectWholeRange(ctx, engine, &diff, request.StartKey, request.EndKey, + tsGC, ms) + ms.Add(diff) + require.Errorf(t, err, "expected error '%s' but found none", expectedError) + require.True(t, testutils.IsError(err, expectedError), + "expected error '%s' found '%s'", expectedError, err) } // TestResolveIntentWithLowerEpoch verifies that trying to resolve @@ -6027,32 +5751,28 @@ func TestResolveIntentWithLowerEpoch(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() + engine := NewDefaultInMemForTesting() + defer engine.Close() - // Lay down an intent with a high epoch. - if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1e2); err != nil { - t.Fatal(err) - } - // Resolve the intent with a low epoch. - if _, err := MVCCResolveWriteIntent(ctx, engine, nil, - roachpb.MakeLockUpdate(txn1, roachpb.Span{Key: testKey1})); err != nil { - t.Fatal(err) - } + // Lay down an intent with a high epoch. + if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, hlc.ClockTimestamp{}, value1, txn1e2); err != nil { + t.Fatal(err) + } + // Resolve the intent with a low epoch. + if _, err := MVCCResolveWriteIntent(ctx, engine, nil, + roachpb.MakeLockUpdate(txn1, roachpb.Span{Key: testKey1})); err != nil { + t.Fatal(err) + } - // Check that the intent was not cleared. - _, intent, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{ - Inconsistent: true, - }) - if err != nil { - t.Fatal(err) - } - if intent == nil { - t.Fatal("intent should not be cleared by resolve intent request with lower epoch") - } - }) + // Check that the intent was not cleared. + _, intent, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{ + Inconsistent: true, + }) + if err != nil { + t.Fatal(err) + } + if intent == nil { + t.Fatal("intent should not be cleared by resolve intent request with lower epoch") } } @@ -6063,36 +5783,32 @@ func TestTimeSeriesMVCCStats(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - var ms = enginepb.MVCCStats{} + engine := NewDefaultInMemForTesting() + defer engine.Close() + var ms = enginepb.MVCCStats{} - // Perform a sequence of merges on the same key - // and record the MVCC stats for it. - if err := MVCCMerge(ctx, engine, &ms, testKey1, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { - t.Fatal(err) - } - firstMS := ms + // Perform a sequence of merges on the same key + // and record the MVCC stats for it. + if err := MVCCMerge(ctx, engine, &ms, testKey1, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { + t.Fatal(err) + } + firstMS := ms - if err := MVCCMerge(ctx, engine, &ms, testKey1, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { - t.Fatal(err) - } - secondMS := ms + if err := MVCCMerge(ctx, engine, &ms, testKey1, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { + t.Fatal(err) + } + secondMS := ms - // Ensure timeseries metrics increase as expected. - expectedMS := firstMS - expectedMS.LiveBytes += int64(len(tsvalue1.RawBytes)) - expectedMS.ValBytes += int64(len(tsvalue1.RawBytes)) + // Ensure timeseries metrics increase as expected. + expectedMS := firstMS + expectedMS.LiveBytes += int64(len(tsvalue1.RawBytes)) + expectedMS.ValBytes += int64(len(tsvalue1.RawBytes)) - if secondMS.LiveBytes != expectedMS.LiveBytes { - t.Fatalf("second merged LiveBytes value %v differed from expected LiveBytes value %v", secondMS.LiveBytes, expectedMS.LiveBytes) - } - if secondMS.ValBytes != expectedMS.ValBytes { - t.Fatalf("second merged ValBytes value %v differed from expected ValBytes value %v", secondMS.LiveBytes, expectedMS.LiveBytes) - } - }) + if secondMS.LiveBytes != expectedMS.LiveBytes { + t.Fatalf("second merged LiveBytes value %v differed from expected LiveBytes value %v", secondMS.LiveBytes, expectedMS.LiveBytes) + } + if secondMS.ValBytes != expectedMS.ValBytes { + t.Fatalf("second merged ValBytes value %v differed from expected ValBytes value %v", secondMS.LiveBytes, expectedMS.LiveBytes) } } @@ -6103,61 +5819,57 @@ func TestMVCCTimeSeriesPartialMerge(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - engine := engineImpl.create() - defer engine.Close() - - // Perform the same sequence of merges on two different keys. For - // one of them, insert some compactions which cause partial merges - // to be run and affect the results. - vals := make([]*roachpb.Value, 2) + engine := NewDefaultInMemForTesting() + defer engine.Close() - for i, k := range []roachpb.Key{testKey1, testKey2} { - if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { - t.Fatal(err) - } - if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 2}, tsvalue2); err != nil { - t.Fatal(err) - } + // Perform the same sequence of merges on two different keys. For + // one of them, insert some compactions which cause partial merges + // to be run and affect the results. + vals := make([]*roachpb.Value, 2) - if i == 1 { - if err := engine.Compact(); err != nil { - t.Fatal(err) - } - } + for i, k := range []roachpb.Key{testKey1, testKey2} { + if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { + t.Fatal(err) + } + if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 2}, tsvalue2); err != nil { + t.Fatal(err) + } - if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 2}, tsvalue2); err != nil { - t.Fatal(err) - } - if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { - t.Fatal(err) - } + if i == 1 { + if err := engine.Compact(); err != nil { + t.Fatal(err) + } + } - if i == 1 { - if err := engine.Compact(); err != nil { - t.Fatal(err) - } - } + if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 2}, tsvalue2); err != nil { + t.Fatal(err) + } + if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { + t.Fatal(err) + } - if v, _, err := MVCCGet(ctx, engine, k, hlc.Timestamp{}, MVCCGetOptions{}); err != nil { - t.Fatal(err) - } else { - vals[i] = v - } + if i == 1 { + if err := engine.Compact(); err != nil { + t.Fatal(err) } + } - if first, second := vals[0], vals[1]; !reflect.DeepEqual(first, second) { - var firstTS, secondTS roachpb.InternalTimeSeriesData - if err := first.GetProto(&firstTS); err != nil { - t.Fatal(err) - } - if err := second.GetProto(&secondTS); err != nil { - t.Fatal(err) - } - t.Fatalf("partially merged value %v differed from expected merged value %v", secondTS, firstTS) - } - }) + if v, _, err := MVCCGet(ctx, engine, k, hlc.Timestamp{}, MVCCGetOptions{}); err != nil { + t.Fatal(err) + } else { + vals[i] = v + } + } + + if first, second := vals[0], vals[1]; !reflect.DeepEqual(first, second) { + var firstTS, secondTS roachpb.InternalTimeSeriesData + if err := first.GetProto(&firstTS); err != nil { + t.Fatal(err) + } + if err := second.GetProto(&secondTS); err != nil { + t.Fatal(err) + } + t.Fatalf("partially merged value %v differed from expected merged value %v", secondTS, firstTS) } } diff --git a/pkg/storage/sst_test.go b/pkg/storage/sst_test.go index b328e62066e0..8e2b50a44f91 100644 --- a/pkg/storage/sst_test.go +++ b/pkg/storage/sst_test.go @@ -62,45 +62,41 @@ func TestCheckSSTConflictsMaxIntents(t *testing.T) { require.NoError(t, sstWriter.Finish()) sstWriter.Close() - for _, engineImpl := range mvccEngineImpls { - t.Run(engineImpl.name, func(t *testing.T) { - ctx := context.Background() - engine := engineImpl.create(Settings(cs)) - defer engine.Close() - - // Write some committed keys and intents at txn1TS. - batch := engine.NewBatch() - for _, key := range keys { - mvccKey := MVCCKey{Key: roachpb.Key(key), Timestamp: txn1TS} - mvccValue := MVCCValue{Value: roachpb.MakeValueFromString("value")} - require.NoError(t, batch.PutMVCC(mvccKey, mvccValue)) - } - for _, key := range intents { - require.NoError(t, MVCCPut(ctx, batch, nil, roachpb.Key(key), txn1TS, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("intent"), txn1)) - } - require.NoError(t, batch.Commit(true)) - batch.Close() - require.NoError(t, engine.Flush()) - - for _, tc := range testcases { - t.Run(fmt.Sprintf("maxIntents=%d", tc.maxIntents), func(t *testing.T) { - for _, usePrefixSeek := range []bool{false, true} { - t.Run(fmt.Sprintf("usePrefixSeek=%v", usePrefixSeek), func(t *testing.T) { - // Provoke and check WriteIntentErrors. - startKey, endKey := MVCCKey{Key: roachpb.Key(start)}, MVCCKey{Key: roachpb.Key(end)} - _, err := CheckSSTConflicts(ctx, sstFile.Bytes(), engine, startKey, endKey, startKey.Key, endKey.Key.Next(), - false /*disallowShadowing*/, hlc.Timestamp{} /*disallowShadowingBelow*/, tc.maxIntents, usePrefixSeek) - require.Error(t, err) - writeIntentErr := &roachpb.WriteIntentError{} - require.ErrorAs(t, err, &writeIntentErr) - - actual := []string{} - for _, i := range writeIntentErr.Intents { - actual = append(actual, string(i.Key)) - } - require.Equal(t, tc.expectIntents, actual) - }) + ctx := context.Background() + engine := NewDefaultInMemForTesting(Settings(cs)) + defer engine.Close() + + // Write some committed keys and intents at txn1TS. + batch := engine.NewBatch() + for _, key := range keys { + mvccKey := MVCCKey{Key: roachpb.Key(key), Timestamp: txn1TS} + mvccValue := MVCCValue{Value: roachpb.MakeValueFromString("value")} + require.NoError(t, batch.PutMVCC(mvccKey, mvccValue)) + } + for _, key := range intents { + require.NoError(t, MVCCPut(ctx, batch, nil, roachpb.Key(key), txn1TS, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("intent"), txn1)) + } + require.NoError(t, batch.Commit(true)) + batch.Close() + require.NoError(t, engine.Flush()) + + for _, tc := range testcases { + t.Run(fmt.Sprintf("maxIntents=%d", tc.maxIntents), func(t *testing.T) { + for _, usePrefixSeek := range []bool{false, true} { + t.Run(fmt.Sprintf("usePrefixSeek=%v", usePrefixSeek), func(t *testing.T) { + // Provoke and check WriteIntentErrors. + startKey, endKey := MVCCKey{Key: roachpb.Key(start)}, MVCCKey{Key: roachpb.Key(end)} + _, err := CheckSSTConflicts(ctx, sstFile.Bytes(), engine, startKey, endKey, startKey.Key, endKey.Key.Next(), + false /*disallowShadowing*/, hlc.Timestamp{} /*disallowShadowingBelow*/, tc.maxIntents, usePrefixSeek) + require.Error(t, err) + writeIntentErr := &roachpb.WriteIntentError{} + require.ErrorAs(t, err, &writeIntentErr) + + actual := []string{} + for _, i := range writeIntentErr.Intents { + actual = append(actual, string(i.Key)) } + require.Equal(t, tc.expectIntents, actual) }) } }) From fa3ebb365b1df7e6ecec9c4623e945c4dd6ed5c6 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Thu, 15 Sep 2022 22:43:05 +0000 Subject: [PATCH 2/3] storage: remove engineRealFSImpls Clean up additional vestiges of the migration from RocksDB to Pebble. Release note: None --- pkg/storage/engine_test.go | 136 +++++++++++++++---------------------- 1 file changed, 55 insertions(+), 81 deletions(-) diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 51144d208042..822f10ac0dfe 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -1213,83 +1213,61 @@ func TestEngineFS(t *testing.T) { } } -type engineImpl struct { - name string - create func(*testing.T, string) Engine -} - -// These FS implementations are not in-memory. -var engineRealFSImpls = []engineImpl{ - {"pebble", func(t *testing.T, dir string) Engine { - db, err := Open( - context.Background(), - Filesystem(dir), - CacheSize(testCacheSize)) - if err != nil { - t.Fatalf("could not create new pebble instance at %s: %+v", dir, err) - } - return db - }}, -} - func TestEngineFSFileNotFoundError(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - for _, engineImpl := range engineRealFSImpls { - t.Run(engineImpl.name, func(t *testing.T) { - dir, dirCleanup := testutils.TempDir(t) - defer dirCleanup() - db := engineImpl.create(t, dir) - defer db.Close() + dir, dirCleanup := testutils.TempDir(t) + defer dirCleanup() + db, err := Open(context.Background(), Filesystem(dir), CacheSize(testCacheSize)) + require.NoError(t, err) + defer db.Close() - // Verify Remove returns os.ErrNotExist if file does not exist. - if err := db.Remove("/non/existent/file"); !oserror.IsNotExist(err) { - t.Fatalf("expected IsNotExist, but got %v (%T)", err, err) - } + // Verify Remove returns os.ErrNotExist if file does not exist. + if err := db.Remove("/non/existent/file"); !oserror.IsNotExist(err) { + t.Fatalf("expected IsNotExist, but got %v (%T)", err, err) + } - // Verify RemoveAll returns nil if path does not exist. - if err := db.RemoveAll("/non/existent/file"); err != nil { - t.Fatalf("expected nil, but got %v (%T)", err, err) - } + // Verify RemoveAll returns nil if path does not exist. + if err := db.RemoveAll("/non/existent/file"); err != nil { + t.Fatalf("expected nil, but got %v (%T)", err, err) + } - fname := filepath.Join(dir, "random.file") - data := "random data" - if f, err := db.Create(fname); err != nil { - t.Fatalf("unable to open file with filename %s, got err %v", fname, err) - } else { - // Write data to file so we can read it later. - if _, err := f.Write([]byte(data)); err != nil { - t.Fatalf("error writing data: '%s' to file %s, got err %v", data, fname, err) - } - if err := f.Sync(); err != nil { - t.Fatalf("error syncing data, got err %v", err) - } - if err := f.Close(); err != nil { - t.Fatalf("error closing file %s, got err %v", fname, err) - } - } + fname := filepath.Join(dir, "random.file") + data := "random data" + if f, err := db.Create(fname); err != nil { + t.Fatalf("unable to open file with filename %s, got err %v", fname, err) + } else { + // Write data to file so we can read it later. + if _, err := f.Write([]byte(data)); err != nil { + t.Fatalf("error writing data: '%s' to file %s, got err %v", data, fname, err) + } + if err := f.Sync(); err != nil { + t.Fatalf("error syncing data, got err %v", err) + } + if err := f.Close(); err != nil { + t.Fatalf("error closing file %s, got err %v", fname, err) + } + } - if b, err := db.ReadFile(fname); err != nil { - t.Errorf("unable to read file with filename %s, got err %v", fname, err) - } else if string(b) != data { - t.Errorf("expected content in %s is '%s', got '%s'", fname, data, string(b)) - } + if b, err := db.ReadFile(fname); err != nil { + t.Errorf("unable to read file with filename %s, got err %v", fname, err) + } else if string(b) != data { + t.Errorf("expected content in %s is '%s', got '%s'", fname, data, string(b)) + } - if err := db.Remove(fname); err != nil { - t.Errorf("unable to delete file with filename %s, got err %v", fname, err) - } + if err := db.Remove(fname); err != nil { + t.Errorf("unable to delete file with filename %s, got err %v", fname, err) + } - // Verify ReadFile returns os.ErrNotExist if reading an already deleted file. - if _, err := db.ReadFile(fname); !oserror.IsNotExist(err) { - t.Fatalf("expected IsNotExist, but got %v (%T)", err, err) - } + // Verify ReadFile returns os.ErrNotExist if reading an already deleted file. + if _, err := db.ReadFile(fname); !oserror.IsNotExist(err) { + t.Fatalf("expected IsNotExist, but got %v (%T)", err, err) + } - // Verify Remove returns os.ErrNotExist if deleting an already deleted file. - if err := db.Remove(fname); !oserror.IsNotExist(err) { - t.Fatalf("expected IsNotExist, but got %v (%T)", err, err) - } - }) + // Verify Remove returns os.ErrNotExist if deleting an already deleted file. + if err := db.Remove(fname); !oserror.IsNotExist(err) { + t.Fatalf("expected IsNotExist, but got %v (%T)", err, err) } } @@ -1326,21 +1304,17 @@ func TestFS(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - var engineImpls []engineImpl - engineImpls = append(engineImpls, engineRealFSImpls...) - engineImpls = append(engineImpls, - engineImpl{ - name: "pebble_mem", - create: func(_ *testing.T, _ string) Engine { - return createTestPebbleEngine() - }, - }) + dir, cleanupDir := testutils.TempDir(t) + defer cleanupDir() - for _, impl := range engineImpls { - t.Run(impl.name, func(t *testing.T) { - dir, cleanupDir := testutils.TempDir(t) - defer cleanupDir() - fs := impl.create(t, dir) + engineDest := map[string]Location{ + "in_memory": InMemory(), + "filesystem": Filesystem(dir), + } + for name, loc := range engineDest { + t.Run(name, func(t *testing.T) { + fs, err := Open(context.Background(), loc, CacheSize(testCacheSize), ForTesting) + require.NoError(t, err) defer fs.Close() path := func(rel string) string { @@ -1370,7 +1344,7 @@ func TestFS(t *testing.T) { expectLS(path("a"), []string{"b"}) expectLS(path("a/b"), []string{"c"}) expectLS(path("a/b/c"), []string{}) - _, err := fs.Stat(path("a/b/c")) + _, err = fs.Stat(path("a/b/c")) require.NoError(t, err) // Create a file at a/b/c/foo. From 8a7538cf665ebdc144aaf153e5297df036371630 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Thu, 15 Sep 2022 22:52:34 +0000 Subject: [PATCH 3/3] storage: remove MVCCIterator.SupportsPrev This interface method is a vestige from RocksDB which did not support Prev on all iterators. Pebble does and all code paths conditional on SupportsPrev may become unconditional. Release note: None --- pkg/kv/kvserver/spanset/batch.go | 5 ---- pkg/storage/engine.go | 3 -- pkg/storage/engine_test.go | 29 ------------------- pkg/storage/intent_interleaving_iter.go | 4 --- pkg/storage/mvcc.go | 10 ++----- .../mvcc_history_metamorphic_iterator_test.go | 15 ++++------ pkg/storage/mvcc_test.go | 14 +-------- pkg/storage/pebble_iterator.go | 5 ---- pkg/storage/point_synthesizing_iter.go | 5 ---- 9 files changed, 8 insertions(+), 82 deletions(-) diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 4895673f9dfa..a691a8684c55 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -220,11 +220,6 @@ func (i *MVCCIterator) IsPrefix() bool { return i.i.IsPrefix() } -// SupportsPrev is part of the storage.MVCCIterator interface. -func (i *MVCCIterator) SupportsPrev() bool { - return i.i.SupportsPrev() -} - // EngineIterator wraps a storage.EngineIterator and ensures that it can // only be used to access spans in a SpanSet. type EngineIterator struct { diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index a7b018037e8f..971b2cddc366 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -279,9 +279,6 @@ type MVCCIterator interface { // IsPrefix returns true if the MVCCIterator is a prefix iterator, i.e. // created with IterOptions.Prefix enabled. IsPrefix() bool - // SupportsPrev returns true if MVCCIterator implementation supports reverse - // iteration with Prev() or SeekLT(). - SupportsPrev() bool } // EngineIterator is an iterator over key-value pairs where the key is diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 822f10ac0dfe..ba28ffaf5bdb 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -1271,35 +1271,6 @@ func TestEngineFSFileNotFoundError(t *testing.T) { } } -// TestSupportPrev tests that SupportsPrev works as expected. -func TestSupportsPrev(t *testing.T) { - defer leaktest.AfterTest(t)() - - eng, err := Open(context.Background(), InMemory(), CacheSize(1<<20 /* 1 MiB */)) - require.NoError(t, err) - defer eng.Close() - opts := IterOptions{LowerBound: keys.LocalMax, UpperBound: keys.MaxKey} - t.Run("engine", func(t *testing.T) { - it := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind, opts) - defer it.Close() - require.Equal(t, true, it.SupportsPrev()) - }) - t.Run("batch", func(t *testing.T) { - batch := eng.NewBatch() - defer batch.Close() - batchIt := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, opts) - defer batchIt.Close() - require.Equal(t, true, batchIt.SupportsPrev()) - }) - t.Run("snapshot", func(t *testing.T) { - snap := eng.NewSnapshot() - defer snap.Close() - snapIt := snap.NewMVCCIterator(MVCCKeyAndIntentsIterKind, opts) - defer snapIt.Close() - require.Equal(t, true, snapIt.SupportsPrev()) - }) -} - func TestFS(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/storage/intent_interleaving_iter.go b/pkg/storage/intent_interleaving_iter.go index 66a2d582c479..b0a116447477 100644 --- a/pkg/storage/intent_interleaving_iter.go +++ b/pkg/storage/intent_interleaving_iter.go @@ -1220,10 +1220,6 @@ func (i *intentInterleavingIter) IsPrefix() bool { return i.prefix } -func (i *intentInterleavingIter) SupportsPrev() bool { - return true -} - // assertInvariants asserts internal iterator invariants, returning an // AssertionFailedf for any violations. It must be called on a valid iterator // after a complete state transition. diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 0d8586902212..625f2a63348a 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -4777,7 +4777,6 @@ func MVCCGarbageCollect( KeyTypes: IterKeyTypePointsAndRanges, }) defer iter.Close() - supportsPrev := iter.SupportsPrev() // Cached stack of range tombstones covering current point. Used to determine // GCBytesAge of deleted value by searching first covering range tombstone @@ -4888,8 +4887,7 @@ func MVCCGarbageCollect( // (key.ts <= gc.ts). var foundPrevNanos bool { - // If reverse iteration is supported (supportsPrev), we'll step the - // iterator a few time before attempting to seek. + // We'll step the iterator a few time before attempting to seek. // True if we found next key while iterating. That means there's no // garbage for the key. @@ -4906,7 +4904,7 @@ func MVCCGarbageCollect( // importantly, this optimization mitigated the overhead of the Seek // approach when almost all of the versions are garbage. const nextsBeforeSeekLT = 4 - for i := 0; !supportsPrev || i < nextsBeforeSeekLT; i++ { + for i := 0; i < nextsBeforeSeekLT; i++ { if i > 0 { iter.Next() } @@ -4942,10 +4940,6 @@ func MVCCGarbageCollect( // its predecessor. Seek to the predecessor to find the right value for // prevNanos and position the iterator on the gcKey. if !foundPrevNanos { - if !supportsPrev { - log.Fatalf(ctx, "failed to find first garbage key without"+ - "support for reverse iteration") - } gcKeyMVCC := MVCCKey{Key: gcKey.Key, Timestamp: gcKey.Timestamp} iter.SeekLT(gcKeyMVCC) if ok, err := iter.Valid(); err != nil { diff --git a/pkg/storage/mvcc_history_metamorphic_iterator_test.go b/pkg/storage/mvcc_history_metamorphic_iterator_test.go index b2ed1b42446a..a3e6a194dfd9 100644 --- a/pkg/storage/mvcc_history_metamorphic_iterator_test.go +++ b/pkg/storage/mvcc_history_metamorphic_iterator_test.go @@ -114,13 +114,12 @@ func (m *metamorphicIterator) moveAround() { func() { m.it.Next() stillValid, _ := m.it.Valid() - if stillValid && mvccIt.SupportsPrev() { + if stillValid { resetActions = append(resetActions, action{ "ResetViaPrev", mvccIt.Prev, }) } - }, }, { @@ -155,7 +154,7 @@ func (m *metamorphicIterator) moveAround() { // Can only leave iterator in reverse mode if it's in reverse // initially, otherwise caller wouldn't be allowed to invoke NextKey // due to MVCCIterator contract. - if !m.isForward && mvccIt.SupportsPrev() { + if !m.isForward { actions = append(actions, action{ "Prev", func() { @@ -234,9 +233,9 @@ func (m *metamorphicIterator) moveAround() { }, }) } - // NB: can't use reverse iteration to find the point if it's not supported, - // or if the iterator is currently forward. - if !m.isForward && hasPoint && mvccIt.SupportsPrev() && !mvccIt.IsPrefix() { + // NB: can't use reverse iteration to find the point if the iterator is + // currently forward. + if !m.isForward && hasPoint && !mvccIt.IsPrefix() { resetActions = append(resetActions, action{ "SeekLT(max) && RevIterate", func() { @@ -393,10 +392,6 @@ func (m *metamorphicMVCCIterator) IsPrefix() bool { return m.it.(MVCCIterator).IsPrefix() } -func (m *metamorphicMVCCIterator) SupportsPrev() bool { - return m.it.(MVCCIterator).SupportsPrev() -} - type metamorphicMVCCIncrementalIterator struct { *metamorphicIterator } diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 23a24529cf85..4b8f07dfa296 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -4803,16 +4803,6 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { toHLC := func(seconds int) hlc.Timestamp { return hlc.Timestamp{WallTime: (time.Duration(seconds) * time.Second).Nanoseconds()} } - engineBatchIteratorSupportsPrev := func(engine Engine) bool { - batch := engine.NewBatch() - defer batch.Close() - it := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ - UpperBound: TestingUserTableDataMin(), - LowerBound: keys.MaxKey, - }) - defer it.Close() - return it.SupportsPrev() - } runTestCase := func(t *testing.T, tc testCase, engine Engine) { ctx := context.Background() ms := &enginepb.MVCCStats{} @@ -4826,8 +4816,6 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { } } - supportsPrev := engineBatchIteratorSupportsPrev(engine) - var keys []roachpb.GCRequest_GCKey var expectedSeekLTs int for _, key := range tc.keys { @@ -4835,7 +4823,7 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { Key: roachpb.Key(key.key), Timestamp: toHLC(key.gcTimestamp), }) - if supportsPrev && key.expSeekLT { + if key.expSeekLT { expectedSeekLTs++ } } diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index 166843e602d7..ae4142241b33 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -908,11 +908,6 @@ func (p *pebbleIterator) IsPrefix() bool { return p.prefix } -// SupportsPrev implements the MVCCIterator interface. -func (p *pebbleIterator) SupportsPrev() bool { - return true -} - // GetRawIter is part of the EngineIterator interface. func (p *pebbleIterator) GetRawIter() *pebble.Iterator { return p.iter diff --git a/pkg/storage/point_synthesizing_iter.go b/pkg/storage/point_synthesizing_iter.go index cc4df96df48d..3377610a7068 100644 --- a/pkg/storage/point_synthesizing_iter.go +++ b/pkg/storage/point_synthesizing_iter.go @@ -685,11 +685,6 @@ func (i *pointSynthesizingIter) IsPrefix() bool { return i.prefix } -// SupportsPrev implements MVCCIterator. -func (i *pointSynthesizingIter) SupportsPrev() bool { - return i.iter.SupportsPrev() -} - // assertInvariants asserts iterator invariants. The iterator must be valid. func (i *pointSynthesizingIter) assertInvariants() error { // Check general MVCCIterator API invariants.