diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index e3ca409b3d20..d99b56820c85 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -141,6 +141,7 @@ go_test( "alter_backup_test.go", "backup_cloud_test.go", "backup_intents_test.go", + "backup_metadata_test.go", "backup_planning_test.go", "backup_rand_test.go", "backup_tenant_test.go", @@ -176,6 +177,7 @@ go_test( "//pkg/ccl/backupccl/backupencryption", "//pkg/ccl/backupccl/backupinfo", "//pkg/ccl/backupccl/backuppb", + "//pkg/ccl/backupccl/backuputils", "//pkg/ccl/kvccl", "//pkg/ccl/multiregionccl", "//pkg/ccl/multiregionccl/multiregionccltestutils", diff --git a/pkg/ccl/backupccl/backupinfo/backup_metadata_test.go b/pkg/ccl/backupccl/backup_metadata_test.go similarity index 99% rename from pkg/ccl/backupccl/backupinfo/backup_metadata_test.go rename to pkg/ccl/backupccl/backup_metadata_test.go index c3d86ee7b2f1..9f71a7acbf06 100644 --- a/pkg/ccl/backupccl/backupinfo/backup_metadata_test.go +++ b/pkg/ccl/backupccl/backup_metadata_test.go @@ -6,7 +6,7 @@ // // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package backupinfo_test +package backupccl_test import ( "context" diff --git a/pkg/ccl/backupccl/backupinfo/BUILD.bazel b/pkg/ccl/backupccl/backupinfo/BUILD.bazel index ddde5478b66d..70c295700665 100644 --- a/pkg/ccl/backupccl/backupinfo/BUILD.bazel +++ b/pkg/ccl/backupccl/backupinfo/BUILD.bazel @@ -48,36 +48,16 @@ go_library( go_test( name = "backupinfo_test", - srcs = [ - "backup_metadata_test.go", - "main_test.go", - ], + srcs = ["main_test.go"], embed = [":backupinfo"], deps = [ - "//pkg/base", - "//pkg/blobs", - "//pkg/ccl/backupccl/backupbase", - "//pkg/ccl/backupccl/backuppb", - "//pkg/ccl/backupccl/backuputils", "//pkg/ccl/utilccl", - "//pkg/cloud", - "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", - "//pkg/security/username", "//pkg/server", - "//pkg/sql", - "//pkg/sql/catalog/descpb", - "//pkg/sql/stats", "//pkg/testutils/serverutils", "//pkg/testutils/testcluster", - "//pkg/util/hlc", - "//pkg/util/ioctx", - "//pkg/util/leaktest", - "//pkg/util/log", - "//pkg/util/protoutil", "//pkg/util/randutil", - "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/ccl/backupccl/backuputils/testutils.go b/pkg/ccl/backupccl/backuputils/testutils.go index e52908d31862..398c2977496c 100644 --- a/pkg/ccl/backupccl/backuputils/testutils.go +++ b/pkg/ccl/backupccl/backuputils/testutils.go @@ -47,6 +47,8 @@ func BackupDestinationTestSetup( return backupRestoreTestSetupWithParams(t, clusterSize, numAccounts, init, base.TestClusterArgs{}) } +// TODO(ssd): This function is very similar a function in backupccl with the same name. +// We should only have 1. func backupRestoreTestSetupWithParams( t testing.TB, clusterSize int, @@ -59,11 +61,13 @@ func backupRestoreTestSetupWithParams( dir, dirCleanupFn := testutils.TempDir(t) params.ServerArgs.ExternalIODir = dir params.ServerArgs.UseDatabase = "data" + params.ServerArgs.DisableDefaultTestTenant = true if len(params.ServerArgsPerNode) > 0 { for i := range params.ServerArgsPerNode { param := params.ServerArgsPerNode[i] param.ExternalIODir = dir param.UseDatabase = "data" + param.DisableDefaultTestTenant = true params.ServerArgsPerNode[i] = param } } diff --git a/pkg/cli/debug_test.go b/pkg/cli/debug_test.go index dd87cff7930f..efdc003be45d 100644 --- a/pkg/cli/debug_test.go +++ b/pkg/cli/debug_test.go @@ -101,6 +101,7 @@ func TestOpenReadOnlyStore(t *testing.T) { func TestRemoveDeadReplicas(t *testing.T) { defer leaktest.AfterTest(t)() + skip.WithIssue(t, 75133, "flaky test") defer log.Scope(t).Close(t) // This test is pretty slow under race (200+ cpu-seconds) because it diff --git a/pkg/kv/kvclient/rangefeed/BUILD.bazel b/pkg/kv/kvclient/rangefeed/BUILD.bazel index 888d3cd3f275..3e090437bb0f 100644 --- a/pkg/kv/kvclient/rangefeed/BUILD.bazel +++ b/pkg/kv/kvclient/rangefeed/BUILD.bazel @@ -77,7 +77,7 @@ go_test( "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", - "//pkg/testutils/sstutil", + "//pkg/testutils/storageutils", "//pkg/testutils/testcluster", "//pkg/util/ctxgroup", "//pkg/util/encoding", diff --git a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go index e9c9673b6082..e548517a6f49 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go @@ -26,7 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/testutils/sstutil" + "github.com/cockroachdb/cockroach/pkg/testutils/storageutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -37,6 +37,10 @@ import ( "github.com/stretchr/testify/require" ) +var pointKV = storageutils.PointKV + +type kvs = storageutils.KVs + // TestRangeFeedIntegration is a basic integration test demonstrating all of // the pieces working together. func TestRangeFeedIntegration(t *testing.T) { @@ -512,9 +516,15 @@ func TestWithOnSSTable(t *testing.T) { // Ingest an SST. now := db.Clock().Now() now.Logical = 0 - ts := now.WallTime - sstKVs := []sstutil.KV{{"a", ts, "1"}, {"b", ts, "2"}, {"c", ts, "3"}, {"d", ts, "4"}, {"e", ts, "5"}} - sst, sstStart, sstEnd := sstutil.MakeSST(t, srv.ClusterSettings(), sstKVs) + ts := int(now.WallTime) + sstKVs := kvs{ + pointKV("a", ts, "1"), + pointKV("b", ts, "2"), + pointKV("c", ts, "3"), + pointKV("d", ts, "4"), + pointKV("e", ts, "5"), + } + sst, sstStart, sstEnd := storageutils.MakeSST(t, srv.ClusterSettings(), sstKVs) _, _, _, pErr := db.AddSSTableAtBatchTimestamp(ctx, sstStart, sstEnd, sst, false /* disallowConflicts */, false /* disallowShadowing */, hlc.Timestamp{}, nil, /* stats */ false /* ingestAsWrites */, now) @@ -530,7 +540,7 @@ func TestWithOnSSTable(t *testing.T) { require.Equal(t, roachpb.Span{Key: sstStart, EndKey: sstEnd}, sstEvent.Span) require.Equal(t, now, sstEvent.WriteTS) - require.Equal(t, sstKVs, sstutil.ScanSST(t, sstEvent.Data)) + require.Equal(t, sstKVs, storageutils.ScanSST(t, sstEvent.Data)) } // TestWithOnSSTableCatchesUpIfNotSet tests that the rangefeed runs a catchup @@ -538,6 +548,8 @@ func TestWithOnSSTable(t *testing.T) { func TestWithOnSSTableCatchesUpIfNotSet(t *testing.T) { defer leaktest.AfterTest(t)() + storage.DisableMetamorphicSimpleValueEncoding(t) + ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -587,10 +599,16 @@ func TestWithOnSSTableCatchesUpIfNotSet(t *testing.T) { // Ingest an SST. now := db.Clock().Now() now.Logical = 0 - ts := now.WallTime - sstKVs := []sstutil.KV{{"a", ts, "1"}, {"b", ts, "2"}, {"c", ts, "3"}, {"d", ts, "4"}, {"e", ts, "5"}} - expectKVs := []sstutil.KV{{"c", ts, "3"}, {"d", ts, "4"}} - sst, sstStart, sstEnd := sstutil.MakeSST(t, srv.ClusterSettings(), sstKVs) + ts := int(now.WallTime) + sstKVs := kvs{ + pointKV("a", ts, "1"), + pointKV("b", ts, "2"), + pointKV("c", ts, "3"), + pointKV("d", ts, "4"), + pointKV("e", ts, "5"), + } + expectKVs := kvs{pointKV("c", ts, "3"), pointKV("d", ts, "4")} + sst, sstStart, sstEnd := storageutils.MakeSST(t, srv.ClusterSettings(), sstKVs) _, _, _, pErr := db.AddSSTableAtBatchTimestamp(ctx, sstStart, sstEnd, sst, false /* disallowConflicts */, false /* disallowShadowing */, hlc.Timestamp{}, nil, /* stats */ false /* ingestAsWrites */, now) @@ -598,16 +616,16 @@ func TestWithOnSSTableCatchesUpIfNotSet(t *testing.T) { // Assert that we receive the KV pairs within the rangefeed span. timer := time.NewTimer(3 * time.Second) - var seenKVs []sstutil.KV + var seenKVs kvs for len(seenKVs) < len(expectKVs) { select { case row := <-rowC: - value, err := row.Value.GetBytes() - require.NoError(t, err) - seenKVs = append(seenKVs, sstutil.KV{ - KeyString: string(row.Key), - WallTimestamp: row.Value.Timestamp.WallTime, - ValueString: string(value), + seenKVs = append(seenKVs, storage.MVCCKeyValue{ + Key: storage.MVCCKey{ + Key: row.Key, + Timestamp: row.Value.Timestamp, + }, + Value: row.Value.RawBytes, }) case <-timer.C: require.Fail(t, "timed out waiting for catchup scan", "saw entries: %v", seenKVs) diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 6ab024da131d..605c4aee269d 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -393,7 +393,7 @@ go_test( "//pkg/testutils/serverutils", "//pkg/testutils/skip", "//pkg/testutils/sqlutils", - "//pkg/testutils/sstutil", + "//pkg/testutils/storageutils", "//pkg/testutils/testcluster", "//pkg/ts", "//pkg/ts/tspb", diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 3b3d43148492..1220bafd9e6c 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -118,6 +118,7 @@ go_test( "declare_test.go", "intent_test.go", "main_test.go", + "testutils_test.go", "transaction_test.go", ], embed = [":batcheval"], @@ -151,7 +152,7 @@ go_test( "//pkg/testutils/serverutils", "//pkg/testutils/skip", "//pkg/testutils/sqlutils", - "//pkg/testutils/sstutil", + "//pkg/testutils/storageutils", "//pkg/testutils/testcluster", "//pkg/util", "//pkg/util/encoding", diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 4f04d14ea39e..bece84fd29ea 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -18,7 +18,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" @@ -30,12 +29,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" - "github.com/cockroachdb/cockroach/pkg/testutils/sstutil" + "github.com/cockroachdb/cockroach/pkg/testutils/storageutils" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" @@ -47,76 +45,78 @@ func TestEvalAddSSTable(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + storage.DisableMetamorphicSimpleValueEncoding(t) + const intentTS = 100 // values with this timestamp are written as intents // These are run with IngestAsWrites both disabled and enabled, and // kv.bulk_io_write.sst_rewrite_concurrency.per_call of 0 and 8. testcases := map[string]struct { - data []sstutil.KV - sst []sstutil.KV + data kvs + sst kvs reqTS int64 toReqTS int64 // SSTTimestampToRequestTimestamp with given SST timestamp noConflict bool // DisallowConflicts noShadow bool // DisallowShadowing noShadowBelow int64 // DisallowShadowingBelow requireReqTS bool // AddSSTableRequireAtRequestTimestamp - expect []sstutil.KV + expect kvs expectErr interface{} // error type, substring, substring slice, or true (any) expectErrRace interface{} expectStatsEst bool // expect MVCCStats.ContainsEstimates, don't check stats }{ // Blind writes. "blind writes below existing": { - data: []sstutil.KV{{"a", 5, "a5"}, {"b", 7, ""}, {"c", 6, "c6"}}, - sst: []sstutil.KV{{"a", 3, "sst"}, {"b", 2, "sst"}, {"c", 3, ""}}, - expect: []sstutil.KV{ - {"a", 5, "a5"}, {"a", 3, "sst"}, {"b", 7, ""}, {"b", 2, "sst"}, {"c", 6, "c6"}, {"c", 3, ""}, + data: kvs{pointKV("a", 5, "a5"), pointKV("b", 7, ""), pointKV("c", 6, "c6")}, + sst: kvs{pointKV("a", 3, "sst"), pointKV("b", 2, "sst"), pointKV("c", 3, "")}, + expect: kvs{ + pointKV("a", 5, "a5"), pointKV("a", 3, "sst"), pointKV("b", 7, ""), pointKV("b", 2, "sst"), pointKV("c", 6, "c6"), pointKV("c", 3, ""), }, expectStatsEst: true, }, "blind replaces existing": { - data: []sstutil.KV{{"a", 2, "a2"}, {"b", 2, "b2"}}, - sst: []sstutil.KV{{"a", 2, "sst"}, {"b", 2, ""}}, - expect: []sstutil.KV{{"a", 2, "sst"}, {"b", 2, ""}}, + data: kvs{pointKV("a", 2, "a2"), pointKV("b", 2, "b2")}, + sst: kvs{pointKV("a", 2, "sst"), pointKV("b", 2, "")}, + expect: kvs{pointKV("a", 2, "sst"), pointKV("b", 2, "")}, expectStatsEst: true, }, "blind errors on AddSSTableRequireAtRequestTimestamp": { - data: []sstutil.KV{{"a", 5, "a5"}, {"b", 7, ""}}, - sst: []sstutil.KV{{"a", 3, "sst"}, {"b", 2, "sst"}}, + data: kvs{pointKV("a", 5, "a5"), pointKV("b", 7, "")}, + sst: kvs{pointKV("a", 3, "sst"), pointKV("b", 2, "sst")}, requireReqTS: true, expectErr: "AddSSTable requests must set SSTTimestampToRequestTimestamp", }, "blind returns WriteIntentError on conflict": { - data: []sstutil.KV{{"b", intentTS, "b0"}}, - sst: []sstutil.KV{{"b", 1, "sst"}}, + data: kvs{pointKV("b", intentTS, "b0")}, + sst: kvs{pointKV("b", 1, "sst")}, expectErr: &roachpb.WriteIntentError{}, }, "blind returns WriteIntentError in span": { - data: []sstutil.KV{{"b", intentTS, "b0"}}, - sst: []sstutil.KV{{"a", 1, "sst"}, {"c", 1, "sst"}}, + data: kvs{pointKV("b", intentTS, "b0")}, + sst: kvs{pointKV("a", 1, "sst"), pointKV("c", 1, "sst")}, expectErr: &roachpb.WriteIntentError{}, }, "blind ignores intent outside span": { - data: []sstutil.KV{{"b", intentTS, "b0"}}, - sst: []sstutil.KV{{"c", 1, "sst"}, {"d", 1, "sst"}}, - expect: []sstutil.KV{{"b", intentTS, "b0"}, {"c", 1, "sst"}, {"d", 1, "sst"}}, + data: kvs{pointKV("b", intentTS, "b0")}, + sst: kvs{pointKV("c", 1, "sst"), pointKV("d", 1, "sst")}, + expect: kvs{pointKV("b", intentTS, "b0"), pointKV("c", 1, "sst"), pointKV("d", 1, "sst")}, expectStatsEst: true, }, "blind writes tombstones": { - sst: []sstutil.KV{{"a", 1, ""}}, - expect: []sstutil.KV{{"a", 1, ""}}, + sst: kvs{pointKV("a", 1, "")}, + expect: kvs{pointKV("a", 1, "")}, expectStatsEst: true, }, "blind writes SST inline values unless race": { // unfortunately, for performance - sst: []sstutil.KV{{"a", 0, "inline"}}, - expect: []sstutil.KV{{"a", 0, "inline"}}, + sst: kvs{pointKV("a", 0, "inline")}, + expect: kvs{pointKV("a", 0, "inline")}, expectStatsEst: true, expectErrRace: `SST contains inline value or intent for key "a"/0,0`, }, "blind writes above existing inline values": { // unfortunately, for performance - data: []sstutil.KV{{"a", 0, "inline"}}, - sst: []sstutil.KV{{"a", 2, "sst"}}, - expect: []sstutil.KV{{"a", 0, "inline"}, {"a", 2, "sst"}}, + data: kvs{pointKV("a", 0, "inline")}, + sst: kvs{pointKV("a", 2, "sst")}, + expect: kvs{pointKV("a", 0, "inline"), pointKV("a", 2, "sst")}, expectStatsEst: true, }, @@ -124,29 +124,29 @@ func TestEvalAddSSTable(t *testing.T) { "SSTTimestampToRequestTimestamp rewrites timestamp": { reqTS: 10, toReqTS: 1, - sst: []sstutil.KV{{"a", 1, "a1"}, {"b", 1, "b1"}}, - expect: []sstutil.KV{{"a", 10, "a1"}, {"b", 10, "b1"}}, + sst: kvs{pointKV("a", 1, "a1"), pointKV("b", 1, "b1")}, + expect: kvs{pointKV("a", 10, "a1"), pointKV("b", 10, "b1")}, expectStatsEst: true, }, "SSTTimestampToRequestTimestamp succeeds on AddSSTableRequireAtRequestTimestamp": { reqTS: 10, toReqTS: 1, requireReqTS: true, - sst: []sstutil.KV{{"a", 1, "a1"}, {"b", 1, "b1"}}, - expect: []sstutil.KV{{"a", 10, "a1"}, {"b", 10, "b1"}}, + sst: kvs{pointKV("a", 1, "a1"), pointKV("b", 1, "b1")}, + expect: kvs{pointKV("a", 10, "a1"), pointKV("b", 10, "b1")}, expectStatsEst: true, }, "SSTTimestampToRequestTimestamp writes tombstones": { reqTS: 10, toReqTS: 1, - sst: []sstutil.KV{{"a", 1, ""}}, - expect: []sstutil.KV{{"a", 10, ""}}, + sst: kvs{pointKV("a", 1, "")}, + expect: kvs{pointKV("a", 10, "")}, expectStatsEst: true, }, "SSTTimestampToRequestTimestamp rejects incorrect SST timestamp": { reqTS: 10, toReqTS: 1, - sst: []sstutil.KV{{"a", 1, "a1"}, {"b", 1, "b1"}, {"c", 2, "c2"}}, + sst: kvs{pointKV("a", 1, "a1"), pointKV("b", 1, "b1"), pointKV("c", 2, "c2")}, expectErr: []string{ `unexpected timestamp 0.000000002,0 (expected 0.000000001,0) for key "c"`, `key has suffix "\x00\x00\x00\x00\x00\x00\x00\x02\t", expected "\x00\x00\x00\x00\x00\x00\x00\x01\t"`, @@ -155,7 +155,7 @@ func TestEvalAddSSTable(t *testing.T) { "SSTTimestampToRequestTimestamp rejects incorrect 0 SST timestamp": { reqTS: 10, toReqTS: 1, - sst: []sstutil.KV{{"a", 1, "a1"}, {"b", 1, "b1"}, {"c", 0, "c0"}}, + sst: kvs{pointKV("a", 1, "a1"), pointKV("b", 1, "b1"), pointKV("c", 0, "c0")}, expectErr: []string{ `unexpected timestamp 0,0 (expected 0.000000001,0) for key "c"`, `key has suffix "", expected "\x00\x00\x00\x00\x00\x00\x00\x01\t"`, @@ -165,274 +165,274 @@ func TestEvalAddSSTable(t *testing.T) { "SSTTimestampToRequestTimestamp writes below and replaces": { reqTS: 5, toReqTS: 1, - data: []sstutil.KV{{"a", 5, "a5"}, {"b", 7, "b7"}}, - sst: []sstutil.KV{{"a", 1, "sst"}, {"b", 1, "sst"}}, - expect: []sstutil.KV{{"a", 5, "sst"}, {"b", 7, "b7"}, {"b", 5, "sst"}}, + data: kvs{pointKV("a", 5, "a5"), pointKV("b", 7, "b7")}, + sst: kvs{pointKV("a", 1, "sst"), pointKV("b", 1, "sst")}, + expect: kvs{pointKV("a", 5, "sst"), pointKV("b", 7, "b7"), pointKV("b", 5, "sst")}, expectStatsEst: true, }, "SSTTimestampToRequestTimestamp returns WriteIntentError for intents": { reqTS: 10, toReqTS: 1, - data: []sstutil.KV{{"a", intentTS, "intent"}}, - sst: []sstutil.KV{{"a", 1, "a@1"}}, + data: kvs{pointKV("a", intentTS, "intent")}, + sst: kvs{pointKV("a", 1, "a@1")}, expectErr: &roachpb.WriteIntentError{}, }, "SSTTimestampToRequestTimestamp errors with DisallowConflicts below existing": { reqTS: 5, toReqTS: 10, noConflict: true, - data: []sstutil.KV{{"a", 5, "a5"}, {"b", 7, "b7"}}, - sst: []sstutil.KV{{"a", 10, "sst"}, {"b", 10, "sst"}}, + data: kvs{pointKV("a", 5, "a5"), pointKV("b", 7, "b7")}, + sst: kvs{pointKV("a", 10, "sst"), pointKV("b", 10, "sst")}, expectErr: &roachpb.WriteTooOldError{}, }, "SSTTimestampToRequestTimestamp succeeds with DisallowConflicts above existing": { reqTS: 8, toReqTS: 1, noConflict: true, - data: []sstutil.KV{{"a", 5, "a5"}, {"b", 7, "b7"}}, - sst: []sstutil.KV{{"a", 1, "sst"}, {"b", 1, "sst"}}, - expect: []sstutil.KV{{"a", 8, "sst"}, {"a", 5, "a5"}, {"b", 8, "sst"}, {"b", 7, "b7"}}, + data: kvs{pointKV("a", 5, "a5"), pointKV("b", 7, "b7")}, + sst: kvs{pointKV("a", 1, "sst"), pointKV("b", 1, "sst")}, + expect: kvs{pointKV("a", 8, "sst"), pointKV("a", 5, "a5"), pointKV("b", 8, "sst"), pointKV("b", 7, "b7")}, }, "SSTTimestampToRequestTimestamp errors with DisallowShadowing below existing": { reqTS: 5, toReqTS: 10, noShadow: true, - data: []sstutil.KV{{"a", 5, "a5"}, {"b", 7, "b7"}}, - sst: []sstutil.KV{{"a", 10, "sst"}, {"b", 10, "sst"}}, + data: kvs{pointKV("a", 5, "a5"), pointKV("b", 7, "b7")}, + sst: kvs{pointKV("a", 10, "sst"), pointKV("b", 10, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "SSTTimestampToRequestTimestamp errors with DisallowShadowing above existing": { reqTS: 8, toReqTS: 1, noShadow: true, - data: []sstutil.KV{{"a", 5, "a5"}, {"b", 7, "b7"}}, - sst: []sstutil.KV{{"a", 1, "sst"}, {"b", 1, "sst"}}, + data: kvs{pointKV("a", 5, "a5"), pointKV("b", 7, "b7")}, + sst: kvs{pointKV("a", 1, "sst"), pointKV("b", 1, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "SSTTimestampToRequestTimestamp succeeds with DisallowShadowing above tombstones": { reqTS: 8, toReqTS: 1, noShadow: true, - data: []sstutil.KV{{"a", 5, ""}, {"b", 7, ""}}, - sst: []sstutil.KV{{"a", 1, "sst"}, {"b", 1, "sst"}}, - expect: []sstutil.KV{{"a", 8, "sst"}, {"a", 5, ""}, {"b", 8, "sst"}, {"b", 7, ""}}, + data: kvs{pointKV("a", 5, ""), pointKV("b", 7, "")}, + sst: kvs{pointKV("a", 1, "sst"), pointKV("b", 1, "sst")}, + expect: kvs{pointKV("a", 8, "sst"), pointKV("a", 5, ""), pointKV("b", 8, "sst"), pointKV("b", 7, "")}, }, "SSTTimestampToRequestTimestamp succeeds with DisallowShadowing and idempotent writes": { reqTS: 5, toReqTS: 1, noShadow: true, - data: []sstutil.KV{{"a", 5, "a5"}, {"b", 5, "b5"}, {"c", 5, ""}}, - sst: []sstutil.KV{{"a", 1, "a5"}, {"b", 1, "b5"}, {"c", 1, ""}}, - expect: []sstutil.KV{{"a", 5, "a5"}, {"b", 5, "b5"}, {"c", 5, ""}}, + data: kvs{pointKV("a", 5, "a5"), pointKV("b", 5, "b5"), pointKV("c", 5, "")}, + sst: kvs{pointKV("a", 1, "a5"), pointKV("b", 1, "b5"), pointKV("c", 1, "")}, + expect: kvs{pointKV("a", 5, "a5"), pointKV("b", 5, "b5"), pointKV("c", 5, "")}, }, "SSTTimestampToRequestTimestamp errors with DisallowShadowingBelow equal value above existing below limit": { reqTS: 7, toReqTS: 10, noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 10, "a3"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 10, "a3")}, expectErr: `ingested key collides with an existing one: "a"`, }, "SSTTimestampToRequestTimestamp errors with DisallowShadowingBelow errors above existing above limit": { reqTS: 7, toReqTS: 10, noShadowBelow: 5, - data: []sstutil.KV{{"a", 6, "a6"}}, - sst: []sstutil.KV{{"a", 10, "sst"}}, + data: kvs{pointKV("a", 6, "a6")}, + sst: kvs{pointKV("a", 10, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "SSTTimestampToRequestTimestamp allows DisallowShadowingBelow equal value above existing above limit": { reqTS: 7, toReqTS: 10, noShadowBelow: 5, - data: []sstutil.KV{{"a", 6, "a6"}}, - sst: []sstutil.KV{{"a", 10, "a6"}}, - expect: []sstutil.KV{{"a", 7, "a6"}, {"a", 6, "a6"}}, + data: kvs{pointKV("a", 6, "a6")}, + sst: kvs{pointKV("a", 10, "a6")}, + expect: kvs{pointKV("a", 7, "a6"), pointKV("a", 6, "a6")}, }, // DisallowConflicts "DisallowConflicts allows above and beside": { noConflict: true, - data: []sstutil.KV{{"a", 3, "a3"}, {"b", 1, ""}}, - sst: []sstutil.KV{{"a", 4, "sst"}, {"b", 3, "sst"}, {"c", 1, "sst"}}, - expect: []sstutil.KV{ - {"a", 4, "sst"}, {"a", 3, "a3"}, {"b", 3, "sst"}, {"b", 1, ""}, {"c", 1, "sst"}, + data: kvs{pointKV("a", 3, "a3"), pointKV("b", 1, "")}, + sst: kvs{pointKV("a", 4, "sst"), pointKV("b", 3, "sst"), pointKV("c", 1, "sst")}, + expect: kvs{ + pointKV("a", 4, "sst"), pointKV("a", 3, "a3"), pointKV("b", 3, "sst"), pointKV("b", 1, ""), pointKV("c", 1, "sst"), }, }, "DisallowConflicts returns WriteTooOldError below existing": { noConflict: true, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 2, "sst"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 2, "sst")}, expectErr: &roachpb.WriteTooOldError{}, }, "DisallowConflicts returns WriteTooOldError at existing": { noConflict: true, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: &roachpb.WriteTooOldError{}, }, "DisallowConflicts returns WriteTooOldError at existing tombstone": { noConflict: true, - data: []sstutil.KV{{"a", 3, ""}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", 3, "")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: &roachpb.WriteTooOldError{}, }, "DisallowConflicts returns WriteIntentError below intent": { noConflict: true, - data: []sstutil.KV{{"a", intentTS, "intent"}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", intentTS, "intent")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: &roachpb.WriteIntentError{}, }, "DisallowConflicts ignores intents in span": { // inconsistent with blind writes noConflict: true, - data: []sstutil.KV{{"b", intentTS, "intent"}}, - sst: []sstutil.KV{{"a", 3, "sst"}, {"c", 3, "sst"}}, - expect: []sstutil.KV{{"a", 3, "sst"}, {"b", intentTS, "intent"}, {"c", 3, "sst"}}, + data: kvs{pointKV("b", intentTS, "intent")}, + sst: kvs{pointKV("a", 3, "sst"), pointKV("c", 3, "sst")}, + expect: kvs{pointKV("a", 3, "sst"), pointKV("b", intentTS, "intent"), pointKV("c", 3, "sst")}, }, "DisallowConflicts is not idempotent": { noConflict: true, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 3, "a3"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 3, "a3")}, expectErr: &roachpb.WriteTooOldError{}, }, "DisallowConflicts allows new SST tombstones": { noConflict: true, - sst: []sstutil.KV{{"a", 3, ""}}, - expect: []sstutil.KV{{"a", 3, ""}}, + sst: kvs{pointKV("a", 3, "")}, + expect: kvs{pointKV("a", 3, "")}, }, "DisallowConflicts allows SST tombstones when shadowing": { noConflict: true, - data: []sstutil.KV{{"a", 2, "a2"}}, - sst: []sstutil.KV{{"a", 3, ""}}, - expect: []sstutil.KV{{"a", 3, ""}, {"a", 2, "a2"}}, + data: kvs{pointKV("a", 2, "a2")}, + sst: kvs{pointKV("a", 3, "")}, + expect: kvs{pointKV("a", 3, ""), pointKV("a", 2, "a2")}, }, "DisallowConflicts allows new SST inline values": { // unfortunately, for performance noConflict: true, - sst: []sstutil.KV{{"a", 0, "inline"}}, - expect: []sstutil.KV{{"a", 0, "inline"}}, + sst: kvs{pointKV("a", 0, "inline")}, + expect: kvs{pointKV("a", 0, "inline")}, expectErrRace: `SST contains inline value or intent for key "a"/0,0`, }, "DisallowConflicts rejects SST inline values when shadowing": { noConflict: true, - data: []sstutil.KV{{"a", 2, "a2"}}, - sst: []sstutil.KV{{"a", 0, ""}}, + data: kvs{pointKV("a", 2, "a2")}, + sst: kvs{pointKV("a", 0, "")}, expectErr: "SST keys must have timestamps", expectErrRace: `SST contains inline value or intent for key "a"/0,0`, }, "DisallowConflicts rejects existing inline values when shadowing": { noConflict: true, - data: []sstutil.KV{{"a", 0, "a0"}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", 0, "a0")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: "inline values are unsupported", }, // DisallowShadowing "DisallowShadowing errors above existing": { noShadow: true, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 4, "sst"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 4, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowing errors below existing": { noShadow: true, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 2, "sst"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 2, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowing errors at existing": { noShadow: true, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowing returns WriteTooOldError at existing tombstone": { noShadow: true, - data: []sstutil.KV{{"a", 3, ""}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", 3, "")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: &roachpb.WriteTooOldError{}, }, "DisallowShadowing returns WriteTooOldError below existing tombstone": { noShadow: true, - data: []sstutil.KV{{"a", 3, ""}}, - sst: []sstutil.KV{{"a", 2, "sst"}}, + data: kvs{pointKV("a", 3, "")}, + sst: kvs{pointKV("a", 2, "sst")}, expectErr: &roachpb.WriteTooOldError{}, }, "DisallowShadowing allows above existing tombstone": { noShadow: true, - data: []sstutil.KV{{"a", 3, ""}}, - sst: []sstutil.KV{{"a", 4, "sst"}}, - expect: []sstutil.KV{{"a", 4, "sst"}, {"a", 3, ""}}, + data: kvs{pointKV("a", 3, "")}, + sst: kvs{pointKV("a", 4, "sst")}, + expect: kvs{pointKV("a", 4, "sst"), pointKV("a", 3, "")}, }, "DisallowShadowing returns WriteIntentError below intent": { noShadow: true, - data: []sstutil.KV{{"a", intentTS, "intent"}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", intentTS, "intent")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: &roachpb.WriteIntentError{}, }, "DisallowShadowing ignores intents in span": { // inconsistent with blind writes noShadow: true, - data: []sstutil.KV{{"b", intentTS, "intent"}}, - sst: []sstutil.KV{{"a", 3, "sst"}, {"c", 3, "sst"}}, - expect: []sstutil.KV{{"a", 3, "sst"}, {"b", intentTS, "intent"}, {"c", 3, "sst"}}, + data: kvs{pointKV("b", intentTS, "intent")}, + sst: kvs{pointKV("a", 3, "sst"), pointKV("c", 3, "sst")}, + expect: kvs{pointKV("a", 3, "sst"), pointKV("b", intentTS, "intent"), pointKV("c", 3, "sst")}, }, "DisallowShadowing is idempotent": { noShadow: true, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 3, "a3"}}, - expect: []sstutil.KV{{"a", 3, "a3"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 3, "a3")}, + expect: kvs{pointKV("a", 3, "a3")}, }, "DisallowShadowing allows new SST tombstones": { noShadow: true, - sst: []sstutil.KV{{"a", 3, ""}}, - expect: []sstutil.KV{{"a", 3, ""}}, + sst: kvs{pointKV("a", 3, "")}, + expect: kvs{pointKV("a", 3, "")}, }, "DisallowShadowing rejects SST tombstones when shadowing": { noShadow: true, - data: []sstutil.KV{{"a", 2, "a2"}}, - sst: []sstutil.KV{{"a", 3, ""}}, + data: kvs{pointKV("a", 2, "a2")}, + sst: kvs{pointKV("a", 3, "")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowing allows new SST inline values": { // unfortunately, for performance noShadow: true, - sst: []sstutil.KV{{"a", 0, "inline"}}, - expect: []sstutil.KV{{"a", 0, "inline"}}, + sst: kvs{pointKV("a", 0, "inline")}, + expect: kvs{pointKV("a", 0, "inline")}, expectErrRace: `SST contains inline value or intent for key "a"/0,0`, }, "DisallowShadowing rejects SST inline values when shadowing": { noShadow: true, - data: []sstutil.KV{{"a", 2, "a2"}}, - sst: []sstutil.KV{{"a", 0, "inline"}}, + data: kvs{pointKV("a", 2, "a2")}, + sst: kvs{pointKV("a", 0, "inline")}, expectErr: "SST keys must have timestamps", expectErrRace: `SST contains inline value or intent for key "a"/0,0`, }, "DisallowShadowing rejects existing inline values when shadowing": { noShadow: true, - data: []sstutil.KV{{"a", 0, "a0"}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", 0, "a0")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: "inline values are unsupported", }, "DisallowShadowing collision SST start, existing start, above": { noShadow: true, - data: []sstutil.KV{{"a", 2, "a2"}}, - sst: []sstutil.KV{{"a", 7, "sst"}}, + data: kvs{pointKV("a", 2, "a2")}, + sst: kvs{pointKV("a", 7, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowing collision SST start, existing middle, below": { noShadow: true, - data: []sstutil.KV{{"a", 2, "a2"}, {"a", 1, "a1"}, {"b", 2, "b2"}, {"c", 3, "c3"}}, - sst: []sstutil.KV{{"b", 1, "sst"}}, + data: kvs{pointKV("a", 2, "a2"), pointKV("a", 1, "a1"), pointKV("b", 2, "b2"), pointKV("c", 3, "c3")}, + sst: kvs{pointKV("b", 1, "sst")}, expectErr: `ingested key collides with an existing one: "b"`, }, "DisallowShadowing collision SST end, existing end, above": { noShadow: true, - data: []sstutil.KV{{"a", 2, "a2"}, {"a", 1, "a1"}, {"b", 2, "b2"}, {"d", 3, "d3"}}, - sst: []sstutil.KV{{"c", 3, "sst"}, {"d", 4, "sst"}}, + data: kvs{pointKV("a", 2, "a2"), pointKV("a", 1, "a1"), pointKV("b", 2, "b2"), pointKV("d", 3, "d3")}, + sst: kvs{pointKV("c", 3, "sst"), pointKV("d", 4, "sst")}, expectErr: `ingested key collides with an existing one: "d"`, }, "DisallowShadowing collision after write above tombstone": { noShadow: true, - data: []sstutil.KV{{"a", 2, ""}, {"a", 1, "a1"}, {"b", 2, "b2"}}, - sst: []sstutil.KV{{"a", 3, "sst"}, {"b", 1, "sst"}}, + data: kvs{pointKV("a", 2, ""), pointKV("a", 1, "a1"), pointKV("b", 2, "b2")}, + sst: kvs{pointKV("a", 3, "sst"), pointKV("b", 1, "sst")}, expectErr: `ingested key collides with an existing one: "b"`, }, @@ -440,240 +440,240 @@ func TestEvalAddSSTable(t *testing.T) { "DisallowShadowingBelow can be used with DisallowShadowing": { noShadow: true, noShadowBelow: 5, - data: []sstutil.KV{{"a", 5, "123"}}, - sst: []sstutil.KV{{"a", 6, "123"}}, - expect: []sstutil.KV{{"a", 6, "123"}, {"a", 5, "123"}}, + data: kvs{pointKV("a", 5, "123")}, + sst: kvs{pointKV("a", 6, "123")}, + expect: kvs{pointKV("a", 6, "123"), pointKV("a", 5, "123")}, }, "DisallowShadowingBelow errors above existing": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 4, "sst"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 4, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow errors below existing": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 2, "sst"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 2, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow errors at existing": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow returns WriteTooOldError at existing tombstone": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, ""}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", 3, "")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: &roachpb.WriteTooOldError{}, }, "DisallowShadowingBelow returns WriteTooOldError below existing tombstone": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, ""}}, - sst: []sstutil.KV{{"a", 2, "sst"}}, + data: kvs{pointKV("a", 3, "")}, + sst: kvs{pointKV("a", 2, "sst")}, expectErr: &roachpb.WriteTooOldError{}, }, "DisallowShadowingBelow allows above existing tombstone": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, ""}}, - sst: []sstutil.KV{{"a", 4, "sst"}}, - expect: []sstutil.KV{{"a", 4, "sst"}, {"a", 3, ""}}, + data: kvs{pointKV("a", 3, "")}, + sst: kvs{pointKV("a", 4, "sst")}, + expect: kvs{pointKV("a", 4, "sst"), pointKV("a", 3, "")}, }, "DisallowShadowingBelow returns WriteIntentError below intent": { noShadowBelow: 5, - data: []sstutil.KV{{"a", intentTS, "intent"}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", intentTS, "intent")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: &roachpb.WriteIntentError{}, }, "DisallowShadowingBelow ignores intents in span": { // inconsistent with blind writes noShadowBelow: 5, - data: []sstutil.KV{{"b", intentTS, "intent"}}, - sst: []sstutil.KV{{"a", 3, "sst"}, {"c", 3, "sst"}}, - expect: []sstutil.KV{{"a", 3, "sst"}, {"b", intentTS, "intent"}, {"c", 3, "sst"}}, + data: kvs{pointKV("b", intentTS, "intent")}, + sst: kvs{pointKV("a", 3, "sst"), pointKV("c", 3, "sst")}, + expect: kvs{pointKV("a", 3, "sst"), pointKV("b", intentTS, "intent"), pointKV("c", 3, "sst")}, }, "DisallowShadowingBelow is not generally idempotent": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 3, "a3"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 3, "a3")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow is not generally idempotent with tombstones": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, ""}}, - sst: []sstutil.KV{{"a", 3, ""}}, + data: kvs{pointKV("a", 3, "")}, + sst: kvs{pointKV("a", 3, "")}, expectErr: &roachpb.WriteTooOldError{}, }, "DisallowShadowingBelow allows new SST tombstones": { noShadowBelow: 5, - sst: []sstutil.KV{{"a", 3, ""}}, - expect: []sstutil.KV{{"a", 3, ""}}, + sst: kvs{pointKV("a", 3, "")}, + expect: kvs{pointKV("a", 3, "")}, }, "DisallowShadowingBelow rejects SST tombstones when shadowing below": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 2, "a2"}}, - sst: []sstutil.KV{{"a", 3, ""}}, + data: kvs{pointKV("a", 2, "a2")}, + sst: kvs{pointKV("a", 3, "")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow allows new SST inline values": { // unfortunately, for performance noShadowBelow: 5, - sst: []sstutil.KV{{"a", 0, "inline"}}, - expect: []sstutil.KV{{"a", 0, "inline"}}, + sst: kvs{pointKV("a", 0, "inline")}, + expect: kvs{pointKV("a", 0, "inline")}, expectErrRace: `SST contains inline value or intent for key "a"/0,0`, }, "DisallowShadowingBelow rejects SST inline values when shadowing": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 2, "a2"}}, - sst: []sstutil.KV{{"a", 0, "inline"}}, + data: kvs{pointKV("a", 2, "a2")}, + sst: kvs{pointKV("a", 0, "inline")}, expectErr: "SST keys must have timestamps", expectErrRace: `SST contains inline value or intent for key "a"/0,0`, }, "DisallowShadowingBelow rejects existing inline values when shadowing": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 0, "a0"}}, - sst: []sstutil.KV{{"a", 3, "sst"}}, + data: kvs{pointKV("a", 0, "a0")}, + sst: kvs{pointKV("a", 3, "sst")}, expectErr: "inline values are unsupported", }, "DisallowShadowingBelow collision SST start, existing start, above": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 2, "a2"}}, - sst: []sstutil.KV{{"a", 7, "sst"}}, + data: kvs{pointKV("a", 2, "a2")}, + sst: kvs{pointKV("a", 7, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow collision SST start, existing middle, below": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 2, "a2"}, {"a", 1, "a1"}, {"b", 2, "b2"}, {"c", 3, "c3"}}, - sst: []sstutil.KV{{"b", 1, "sst"}}, + data: kvs{pointKV("a", 2, "a2"), pointKV("a", 1, "a1"), pointKV("b", 2, "b2"), pointKV("c", 3, "c3")}, + sst: kvs{pointKV("b", 1, "sst")}, expectErr: `ingested key collides with an existing one: "b"`, }, "DisallowShadowingBelow collision SST end, existing end, above": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 2, "a2"}, {"a", 1, "a1"}, {"b", 2, "b2"}, {"d", 3, "d3"}}, - sst: []sstutil.KV{{"c", 3, "sst"}, {"d", 4, "sst"}}, + data: kvs{pointKV("a", 2, "a2"), pointKV("a", 1, "a1"), pointKV("b", 2, "b2"), pointKV("d", 3, "d3")}, + sst: kvs{pointKV("c", 3, "sst"), pointKV("d", 4, "sst")}, expectErr: `ingested key collides with an existing one: "d"`, }, "DisallowShadowingBelow collision after write above tombstone": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 2, ""}, {"a", 1, "a1"}, {"b", 2, "b2"}}, - sst: []sstutil.KV{{"a", 3, "sst"}, {"b", 1, "sst"}}, + data: kvs{pointKV("a", 2, ""), pointKV("a", 1, "a1"), pointKV("b", 2, "b2")}, + sst: kvs{pointKV("a", 3, "sst"), pointKV("b", 1, "sst")}, expectErr: `ingested key collides with an existing one: "b"`, }, "DisallowShadowingBelow tombstone above tombstone": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 2, ""}, {"a", 1, "a1"}}, - sst: []sstutil.KV{{"a", 3, ""}}, - expect: []sstutil.KV{{"a", 3, ""}, {"a", 2, ""}, {"a", 1, "a1"}}, + data: kvs{pointKV("a", 2, ""), pointKV("a", 1, "a1")}, + sst: kvs{pointKV("a", 3, "")}, + expect: kvs{pointKV("a", 3, ""), pointKV("a", 2, ""), pointKV("a", 1, "a1")}, }, "DisallowShadowingBelow at limit writes": { noShadowBelow: 5, - sst: []sstutil.KV{{"a", 5, "sst"}}, - expect: []sstutil.KV{{"a", 5, "sst"}}, + sst: kvs{pointKV("a", 5, "sst")}, + expect: kvs{pointKV("a", 5, "sst")}, }, "DisallowShadowingBelow at limit errors above existing": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 5, "sst"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 5, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow at limit errors above existing with same value": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 3, "a3"}}, - sst: []sstutil.KV{{"a", 5, "a3"}}, + data: kvs{pointKV("a", 3, "a3")}, + sst: kvs{pointKV("a", 5, "a3")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow at limit errors on replacing": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 5, "a3"}}, - sst: []sstutil.KV{{"a", 5, "sst"}}, + data: kvs{pointKV("a", 5, "a3")}, + sst: kvs{pointKV("a", 5, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow at limit is idempotent": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 5, "a3"}}, - sst: []sstutil.KV{{"a", 5, "a3"}}, - expect: []sstutil.KV{{"a", 5, "a3"}}, + data: kvs{pointKV("a", 5, "a3")}, + sst: kvs{pointKV("a", 5, "a3")}, + expect: kvs{pointKV("a", 5, "a3")}, }, "DisallowShadowingBelow above limit writes": { noShadowBelow: 5, - sst: []sstutil.KV{{"a", 7, "sst"}}, - expect: []sstutil.KV{{"a", 7, "sst"}}, + sst: kvs{pointKV("a", 7, "sst")}, + expect: kvs{pointKV("a", 7, "sst")}, }, "DisallowShadowingBelow above limit errors on existing below limit": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 4, "a4"}}, - sst: []sstutil.KV{{"a", 7, "sst"}}, + data: kvs{pointKV("a", 4, "a4")}, + sst: kvs{pointKV("a", 7, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow tombstone above limit errors on existing below limit": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 4, "a4"}}, - sst: []sstutil.KV{{"a", 7, ""}}, + data: kvs{pointKV("a", 4, "a4")}, + sst: kvs{pointKV("a", 7, "")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow above limit errors on existing below limit with same value": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 4, "a4"}}, - sst: []sstutil.KV{{"a", 7, "a3"}}, + data: kvs{pointKV("a", 4, "a4")}, + sst: kvs{pointKV("a", 7, "a3")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow above limit errors on existing at limit": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 5, "a5"}}, - sst: []sstutil.KV{{"a", 7, "sst"}}, + data: kvs{pointKV("a", 5, "a5")}, + sst: kvs{pointKV("a", 7, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow above limit allows equal value at limit": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 5, "a5"}}, - sst: []sstutil.KV{{"a", 7, "a5"}}, - expect: []sstutil.KV{{"a", 7, "a5"}, {"a", 5, "a5"}}, + data: kvs{pointKV("a", 5, "a5")}, + sst: kvs{pointKV("a", 7, "a5")}, + expect: kvs{pointKV("a", 7, "a5"), pointKV("a", 5, "a5")}, }, "DisallowShadowingBelow above limit errors on existing above limit": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 6, "a6"}}, - sst: []sstutil.KV{{"a", 7, "sst"}}, + data: kvs{pointKV("a", 6, "a6")}, + sst: kvs{pointKV("a", 7, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow above limit allows equal value above limit": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 6, "a6"}}, - sst: []sstutil.KV{{"a", 7, "a6"}}, - expect: []sstutil.KV{{"a", 7, "a6"}, {"a", 6, "a6"}}, + data: kvs{pointKV("a", 6, "a6")}, + sst: kvs{pointKV("a", 7, "a6")}, + expect: kvs{pointKV("a", 7, "a6"), pointKV("a", 6, "a6")}, }, "DisallowShadowingBelow above limit errors on replacing": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 7, "a7"}}, - sst: []sstutil.KV{{"a", 7, "sst"}}, + data: kvs{pointKV("a", 7, "a7")}, + sst: kvs{pointKV("a", 7, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow above limit is idempotent": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 7, "a7"}}, - sst: []sstutil.KV{{"a", 7, "a7"}}, - expect: []sstutil.KV{{"a", 7, "a7"}}, + data: kvs{pointKV("a", 7, "a7")}, + sst: kvs{pointKV("a", 7, "a7")}, + expect: kvs{pointKV("a", 7, "a7")}, }, "DisallowShadowingBelow above limit is idempotent with tombstone": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 7, ""}}, - sst: []sstutil.KV{{"a", 7, ""}}, - expect: []sstutil.KV{{"a", 7, ""}}, + data: kvs{pointKV("a", 7, "")}, + sst: kvs{pointKV("a", 7, "")}, + expect: kvs{pointKV("a", 7, "")}, }, "DisallowShadowingBelow above limit errors below existing": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 8, "a8"}}, - sst: []sstutil.KV{{"a", 7, "sst"}}, + data: kvs{pointKV("a", 8, "a8")}, + sst: kvs{pointKV("a", 7, "sst")}, expectErr: `ingested key collides with an existing one: "a"`, }, "DisallowShadowingBelow above limit errors below existing with same value": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 8, "a8"}}, - sst: []sstutil.KV{{"a", 7, "a8"}}, + data: kvs{pointKV("a", 8, "a8")}, + sst: kvs{pointKV("a", 7, "a8")}, expectErr: &roachpb.WriteTooOldError{}, }, "DisallowShadowingBelow above limit errors below tombstone": { noShadowBelow: 5, - data: []sstutil.KV{{"a", 8, ""}}, - sst: []sstutil.KV{{"a", 7, "a8"}}, + data: kvs{pointKV("a", 8, "")}, + sst: kvs{pointKV("a", 7, "a8")}, expectErr: &roachpb.WriteTooOldError{}, }, } @@ -695,21 +695,27 @@ func TestEvalAddSSTable(t *testing.T) { intentTxn := roachpb.MakeTransaction("intentTxn", nil, 0, hlc.Timestamp{WallTime: intentTS}, 0, 1) b := engine.NewBatch() for i := len(tc.data) - 1; i >= 0; i-- { // reverse, older timestamps first - kv := tc.data[i] - var txn *roachpb.Transaction - if kv.WallTimestamp == intentTS { - txn = &intentTxn + switch kv := tc.data[i].(type) { + case storage.MVCCKeyValue: + var txn *roachpb.Transaction + if kv.Key.Timestamp.WallTime == intentTS { + txn = &intentTxn + } + v, err := storage.DecodeMVCCValue(kv.Value) + require.NoError(t, err) + require.NoError(t, storage.MVCCPut(ctx, b, nil, kv.Key.Key, kv.Key.Timestamp, hlc.ClockTimestamp{}, v.Value, txn)) + default: + t.Fatalf("unknown KV type %T", kv) } - require.NoError(t, storage.MVCCPut(ctx, b, nil, kv.Key(), kv.Timestamp(), hlc.ClockTimestamp{}, kv.Value(), txn)) } require.NoError(t, b.Commit(false)) - stats := engineStats(t, engine, 0) + stats := storageutils.EngineStats(t, engine, 0) // Build and add SST. if tc.toReqTS != 0 && tc.reqTS == 0 && tc.expectErr == nil { t.Fatal("can't set toReqTS without reqTS") } - sst, start, end := sstutil.MakeSST(t, st, tc.sst) + sst, start, end := storageutils.MakeSST(t, st, tc.sst) resp := &roachpb.AddSSTableResponse{} var mvccStats *enginepb.MVCCStats // In the no-overlap case i.e. approxDiskBytes == 0, force a regular @@ -719,7 +725,7 @@ func TestEvalAddSSTable(t *testing.T) { // prefix seeks since the test cases have too few keys in the // sstable. if approxDiskBytes != 0 { - mvccStats = sstutil.ComputeStats(t, sst) + mvccStats = storageutils.SSTStats(t, sst, 0) } result, err := batcheval.EvalAddSSTable(ctx, engine, batcheval.CommandArgs{ EvalCtx: (&batcheval.MockEvalCtx{ClusterSettings: st, Desc: &roachpb.RangeDescriptor{}, ApproxDiskBytes: approxDiskBytes}).EvalContext(), @@ -778,47 +784,14 @@ func TestEvalAddSSTable(t *testing.T) { } // Scan resulting data from engine. - iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ - UpperBound: keys.MaxKey, - }) - defer iter.Close() - scan := []sstutil.KV{} - for iter.SeekGE(storage.MVCCKey{Key: keys.SystemPrefix}); ; iter.Next() { - ok, err := iter.Valid() - require.NoError(t, err) - if !ok { - break - } - key := string(iter.Key().Key) - ts := iter.Key().Timestamp.WallTime - var value []byte - if iter.Key().IsValue() { - mvccVal, err := storage.DecodeMVCCValue(iter.Value()) - require.NoError(t, err) - if !mvccVal.IsTombstone() { - value, err = mvccVal.Value.GetBytes() - require.NoError(t, err) - } - } else { - var meta enginepb.MVCCMetadata - require.NoError(t, protoutil.Unmarshal(iter.UnsafeValue(), &meta)) - if meta.RawBytes == nil { - // Skip intent metadata records (value emitted separately). - continue - } - value, err = roachpb.Value{RawBytes: meta.RawBytes}.GetBytes() - require.NoError(t, err) - } - scan = append(scan, sstutil.KV{key, ts, string(value)}) - } - require.Equal(t, tc.expect, scan) + require.Equal(t, tc.expect, storageutils.ScanEngine(t, engine)) // Check that stats were updated correctly. if tc.expectStatsEst { require.NotZero(t, stats.ContainsEstimates, "expected stats to be estimated") } else { require.Zero(t, stats.ContainsEstimates, "found estimated stats") - require.Equal(t, engineStats(t, engine, stats.LastUpdateNanos), stats) + require.Equal(t, storageutils.EngineStats(t, engine, stats.LastUpdateNanos), stats) } }) } @@ -836,31 +809,31 @@ func TestEvalAddSSTableRangefeed(t *testing.T) { reqTS := hlc.Timestamp{WallTime: 10} testcases := map[string]struct { - sst []sstutil.KV + sst kvs toReqTS int64 // SSTTimestampToRequestTimestamp asWrites bool // IngestAsWrites expectHistoryMutation bool expectLogicalOps []enginepb.MVCCLogicalOp }{ "Default": { - sst: []sstutil.KV{{"a", 1, "a1"}}, + sst: kvs{pointKV("a", 1, "a1")}, expectHistoryMutation: true, expectLogicalOps: nil, }, "SSTTimestampToRequestTimestamp alone": { - sst: []sstutil.KV{{"a", 1, "a1"}}, + sst: kvs{pointKV("a", 1, "a1")}, toReqTS: 1, expectHistoryMutation: false, expectLogicalOps: nil, }, "IngestAsWrites alone": { - sst: []sstutil.KV{{"a", 1, "a1"}}, + sst: kvs{pointKV("a", 1, "a1")}, asWrites: true, expectHistoryMutation: true, expectLogicalOps: nil, }, "IngestAsWrites and SSTTimestampToRequestTimestamp": { - sst: []sstutil.KV{{"a", 1, "a1"}, {"b", 1, "b1"}}, + sst: kvs{pointKV("a", 1, "a1"), pointKV("b", 1, "b1")}, asWrites: true, toReqTS: 1, expectHistoryMutation: false, @@ -883,7 +856,7 @@ func TestEvalAddSSTableRangefeed(t *testing.T) { opLogger := storage.NewOpLoggerBatch(engine.NewBatch()) // Build and add SST. - sst, start, end := sstutil.MakeSST(t, st, tc.sst) + sst, start, end := storageutils.MakeSST(t, st, tc.sst) result, err := batcheval.EvalAddSSTable(ctx, opLogger, batcheval.CommandArgs{ EvalCtx: (&batcheval.MockEvalCtx{ClusterSettings: st, Desc: &roachpb.RangeDescriptor{}}).EvalContext(), Header: roachpb.Header{ @@ -893,7 +866,7 @@ func TestEvalAddSSTableRangefeed(t *testing.T) { Args: &roachpb.AddSSTableRequest{ RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, Data: sst, - MVCCStats: sstutil.ComputeStats(t, sst), + MVCCStats: storageutils.SSTStats(t, sst, 0), SSTTimestampToRequestTimestamp: hlc.Timestamp{WallTime: tc.toReqTS}, IngestAsWrites: tc.asWrites, }, @@ -965,7 +938,7 @@ func runTestDBAddSSTable( cs := cluster.MakeTestingClusterSettings() { - sst, start, end := sstutil.MakeSST(t, cs, []sstutil.KV{{"bb", 2, "1"}}) + sst, start, end := storageutils.MakeSST(t, cs, kvs{pointKV("bb", 2, "1")}) // Key is before the range in the request span. _, _, err := db.AddSSTable( @@ -1008,7 +981,7 @@ func runTestDBAddSSTable( // Check that ingesting a key with an earlier mvcc timestamp doesn't affect // the value returned by Get. { - sst, start, end := sstutil.MakeSST(t, cs, []sstutil.KV{{"bb", 1, "2"}}) + sst, start, end := storageutils.MakeSST(t, cs, kvs{pointKV("bb", 1, "2")}) _, _, err := db.AddSSTable( ctx, start, end, sst, allowConflicts, allowShadowing, allowShadowingBelow, nilStats, ingestAsSST, noTS) require.NoError(t, err) @@ -1023,7 +996,7 @@ func runTestDBAddSSTable( // Key range in request span is not empty. First time through a different // key is present. Second time through checks the idempotency. { - sst, start, end := sstutil.MakeSST(t, cs, []sstutil.KV{{"bc", 1, "3"}}) + sst, start, end := storageutils.MakeSST(t, cs, kvs{pointKV("bc", 1, "3")}) var before int64 if store != nil { @@ -1060,7 +1033,7 @@ func runTestDBAddSSTable( // ... and doing the same thing but via write-batch works the same. { - sst, start, end := sstutil.MakeSST(t, cs, []sstutil.KV{{"bd", 1, "3"}}) + sst, start, end := storageutils.MakeSST(t, cs, kvs{pointKV("bd", 1, "3")}) var before int64 if store != nil { @@ -1127,30 +1100,30 @@ func TestAddSSTableMVCCStats(t *testing.T) { engine := storage.NewDefaultInMemForTesting() defer engine.Close() - for _, kv := range []sstutil.KV{ - {"A", 1, "A"}, - {"a", 1, "a"}, - {"a", 6, ""}, - {"b", 5, "bb"}, - {"c", 6, "ccccccccccccccccccccccccccccccccccccccccccccc"}, // key 4b, 50b, live 64b - {"d", 1, "d"}, - {"d", 2, ""}, - {"e", 1, "e"}, - {"u", 3, "u"}, - {"z", 2, "zzzzzz"}, + for _, kv := range []storage.MVCCKeyValue{ + pointKV("A", 1, "A"), + pointKV("a", 1, "a"), + pointKV("a", 6, ""), + pointKV("b", 5, "bb"), + pointKV("c", 6, "ccccccccccccccccccccccccccccccccccccccccccccc"), // key 4b, 50b, live 64b + pointKV("d", 1, "d"), + pointKV("d", 2, ""), + pointKV("e", 1, "e"), + pointKV("u", 3, "u"), + pointKV("z", 2, "zzzzzz"), } { - require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) + require.NoError(t, engine.PutRawMVCC(kv.Key, kv.Value)) } - sst, start, end := sstutil.MakeSST(t, st, []sstutil.KV{ - {"a", 4, "aaaaaa"}, // mvcc-shadowed by existing delete. - {"a", 2, "aa"}, // mvcc-shadowed within SST. - {"c", 6, "ccc"}, // same TS as existing, LSM-shadows existing. - {"d", 4, "dddd"}, // mvcc-shadow existing deleted d. - {"e", 4, "eeee"}, // mvcc-shadow existing 1b. - {"j", 2, "jj"}, // no colission – via MVCC or LSM – with existing. - {"t", 3, ""}, // tombstone, no collission - {"u", 5, ""}, // tombstone, shadows existing + sst, start, end := storageutils.MakeSST(t, st, kvs{ + pointKV("a", 4, "aaaaaa"), // mvcc-shadowed by existing delete. + pointKV("a", 2, "aa"), // mvcc-shadowed within SST. + pointKV("c", 6, "ccc"), // same TS as existing, LSM-shadows existing. + pointKV("d", 4, "dddd"), // mvcc-shadow existing deleted d. + pointKV("e", 4, "eeee"), // mvcc-shadow existing 1b. + pointKV("j", 2, "jj"), // no colission – via MVCC or LSM – with existing. + pointKV("t", 3, ""), // tombstone, no collission + pointKV("u", 5, ""), // tombstone, shadows existing }) statsDelta := enginepb.MVCCStats{ // the sst will think it added 5 keys here, but a, c, e, and t shadow or are shadowed. @@ -1168,7 +1141,7 @@ func TestAddSSTableMVCCStats(t *testing.T) { // After EvalAddSSTable, cArgs.Stats contains a diff to the existing // stats. Make sure recomputing from scratch gets the same answer as // applying the diff to the stats - statsBefore := engineStats(t, engine, 0) + statsBefore := storageutils.EngineStats(t, engine, 0) ts := hlc.Timestamp{WallTime: 7} evalCtx.Stats = *statsBefore @@ -1195,7 +1168,7 @@ func TestAddSSTableMVCCStats(t *testing.T) { statsEvaled.Add(statsDelta) statsEvaled.ContainsEstimates = 0 - newStats := engineStats(t, engine, statsEvaled.LastUpdateNanos) + newStats := storageutils.EngineStats(t, engine, statsEvaled.LastUpdateNanos) require.Equal(t, newStats, statsEvaled) // Check that actual remaining bytes equals the returned remaining bytes once @@ -1203,7 +1176,7 @@ func TestAddSSTableMVCCStats(t *testing.T) { require.Equal(t, max-newStats.Total(), resp.AvailableBytes-statsDelta.Total()) // Check stats for a single KV. - sst, start, end = sstutil.MakeSST(t, st, []sstutil.KV{{"zzzzzzz", ts.WallTime, "zzz"}}) + sst, start, end = storageutils.MakeSST(t, st, kvs{pointKV("zzzzzzz", int(ts.WallTime), "zzz")}) cArgs = batcheval.CommandArgs{ EvalCtx: evalCtx.EvalContext(), Header: roachpb.Header{Timestamp: ts}, @@ -1241,19 +1214,19 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { engine := storage.NewDefaultInMemForTesting() defer engine.Close() - for _, kv := range []sstutil.KV{ - {"a", 2, "aa"}, - {"b", 1, "bb"}, - {"b", 6, ""}, - {"g", 5, "gg"}, - {"r", 1, "rr"}, - {"t", 3, ""}, - {"y", 1, "yy"}, - {"y", 2, ""}, - {"y", 5, "yyy"}, - {"z", 2, "zz"}, + for _, kv := range []storage.MVCCKeyValue{ + pointKV("a", 2, "aa"), + pointKV("b", 1, "bb"), + pointKV("b", 6, ""), + pointKV("g", 5, "gg"), + pointKV("r", 1, "rr"), + pointKV("t", 3, ""), + pointKV("y", 1, "yy"), + pointKV("y", 2, ""), + pointKV("y", 5, "yyy"), + pointKV("z", 2, "zz"), } { - require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) + require.NoError(t, engine.PutRawMVCC(kv.Key, kv.Value)) } // This test ensures accuracy of MVCCStats in the situation that successive @@ -1265,11 +1238,11 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { // CommandArgs Stats field by using: // cArgs.Stats + ingested_stats - skipped_stats. // Successfully evaluate the first SST as there are no key collisions. - kvs := []sstutil.KV{ - {"c", 2, "bb"}, - {"h", 6, "hh"}, + sstKVs := kvs{ + pointKV("c", 2, "bb"), + pointKV("h", 6, "hh"), } - sst, start, end := sstutil.MakeSST(t, st, kvs) + sst, start, end := storageutils.MakeSST(t, st, sstKVs) // Accumulate stats across SST ingestion. commandStats := enginepb.MVCCStats{} @@ -1283,7 +1256,7 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, Data: sst, DisallowShadowing: true, - MVCCStats: sstutil.ComputeStats(t, sst), + MVCCStats: storageutils.SSTStats(t, sst, 0), }, Stats: &commandStats, } @@ -1294,22 +1267,22 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { // Insert KV entries so that we can correctly identify keys to skip when // ingesting the perfectly shadowing KVs (same ts and same value) in the // second SST. - for _, kv := range kvs { - require.NoError(t, engine.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) + for _, kv := range sstKVs.MVCCKeyValues() { + require.NoError(t, engine.PutRawMVCC(kv.Key, kv.Value)) } // Evaluate the second SST. Both the KVs are perfectly shadowing and should // not contribute to the stats. - sst, start, end = sstutil.MakeSST(t, st, []sstutil.KV{ - {"c", 2, "bb"}, // key has the same timestamp and value as the one present in the existing data. - {"h", 6, "hh"}, // key has the same timestamp and value as the one present in the existing data. + sst, start, end = storageutils.MakeSST(t, st, kvs{ + pointKV("c", 2, "bb"), // key has the same timestamp and value as the one present in the existing data. + pointKV("h", 6, "hh"), // key has the same timestamp and value as the one present in the existing data. }) cArgs.Args = &roachpb.AddSSTableRequest{ RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, Data: sst, DisallowShadowing: true, - MVCCStats: sstutil.ComputeStats(t, sst), + MVCCStats: storageutils.SSTStats(t, sst, 0), } _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, &roachpb.AddSSTableResponse{}) require.NoError(t, err) @@ -1319,19 +1292,19 @@ func TestAddSSTableMVCCStatsDisallowShadowing(t *testing.T) { // Evaluate the third SST. Some of the KVs are perfectly shadowing, but there // are two valid KVs which should contribute to the stats. - sst, start, end = sstutil.MakeSST(t, st, []sstutil.KV{ - {"c", 2, "bb"}, // key has the same timestamp and value as the one present in the existing data. - {"e", 2, "ee"}, - {"h", 6, "hh"}, // key has the same timestamp and value as the one present in the existing data. - {"t", 3, ""}, // identical to existing tombstone. - {"x", 7, ""}, // new tombstone. + sst, start, end = storageutils.MakeSST(t, st, kvs{ + pointKV("c", 2, "bb"), // key has the same timestamp and value as the one present in the existing data. + pointKV("e", 2, "ee"), + pointKV("h", 6, "hh"), // key has the same timestamp and value as the one present in the existing data. + pointKV("t", 3, ""), // identical to existing tombstone. + pointKV("x", 7, ""), // new tombstone. }) cArgs.Args = &roachpb.AddSSTableRequest{ RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, Data: sst, DisallowShadowing: true, - MVCCStats: sstutil.ComputeStats(t, sst), + MVCCStats: storageutils.SSTStats(t, sst, 0), } _, err = batcheval.EvalAddSSTable(ctx, engine, cArgs, &roachpb.AddSSTableResponse{}) require.NoError(t, err) @@ -1370,10 +1343,10 @@ func TestAddSSTableIntentResolution(t *testing.T) { // Generate an SSTable that covers keys a, b, and c, and submit it with high // priority. This is going to abort the transaction above, encounter its // intent, and resolve it. - sst, start, end := sstutil.MakeSST(t, s.ClusterSettings(), []sstutil.KV{ - {"a", 1, "1"}, - {"b", 1, "2"}, - {"c", 1, "3"}, + sst, start, end := storageutils.MakeSST(t, s.ClusterSettings(), kvs{ + pointKV("a", 1, "1"), + pointKV("b", 1, "2"), + pointKV("c", 1, "3"), }) ba := roachpb.BatchRequest{ Header: roachpb.Header{UserPriority: roachpb.MaxUserPriority}, @@ -1381,7 +1354,7 @@ func TestAddSSTableIntentResolution(t *testing.T) { ba.Add(&roachpb.AddSSTableRequest{ RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, Data: sst, - MVCCStats: sstutil.ComputeStats(t, sst), + MVCCStats: storageutils.SSTStats(t, sst, 0), DisallowShadowing: true, }) _, pErr := db.NonTransactionalSender().Send(ctx, ba) @@ -1412,11 +1385,11 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsTSCache(t *testing.T) { txnTS := txn.CommitTimestamp() // Add an SST writing below the previous write. - sst, start, end := sstutil.MakeSST(t, s.ClusterSettings(), []sstutil.KV{{"key", 1, "sst"}}) + sst, start, end := storageutils.MakeSST(t, s.ClusterSettings(), kvs{pointKV("key", 1, "sst")}) sstReq := &roachpb.AddSSTableRequest{ RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, Data: sst, - MVCCStats: sstutil.ComputeStats(t, sst), + MVCCStats: storageutils.SSTStats(t, sst, 0), SSTTimestampToRequestTimestamp: hlc.Timestamp{WallTime: 1}, } ba := roachpb.BatchRequest{ @@ -1472,11 +1445,11 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsClosedTS(t *testing.T) // Add an SST writing below the closed timestamp. It should get pushed above it. reqTS := closedTS.Prev() - sst, start, end := sstutil.MakeSST(t, store.ClusterSettings(), []sstutil.KV{{"key", 1, "sst"}}) + sst, start, end := storageutils.MakeSST(t, store.ClusterSettings(), kvs{pointKV("key", 1, "sst")}) sstReq := &roachpb.AddSSTableRequest{ RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, Data: sst, - MVCCStats: sstutil.ComputeStats(t, sst), + MVCCStats: storageutils.SSTStats(t, sst, 0), SSTTimestampToRequestTimestamp: hlc.Timestamp{WallTime: 1}, } ba := roachpb.BatchRequest{ @@ -1500,18 +1473,3 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsClosedTS(t *testing.T) require.NoError(t, err) require.Equal(t, "sst", string(v)) } - -// engineStats computes the MVCC stats for the given engine. -func engineStats(t *testing.T, engine storage.Engine, nowNanos int64) *enginepb.MVCCStats { - t.Helper() - - iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ - KeyTypes: storage.IterKeyTypePointsAndRanges, - LowerBound: keys.LocalMax, - UpperBound: keys.MaxKey, - }) - defer iter.Close() - stats, err := storage.ComputeStatsForRange(iter, keys.LocalMax, keys.MaxKey, nowNanos) - require.NoError(t, err) - return &stats -} diff --git a/pkg/kv/kvserver/batcheval/testutils_test.go b/pkg/kv/kvserver/batcheval/testutils_test.go new file mode 100644 index 000000000000..b91cbd64801f --- /dev/null +++ b/pkg/kv/kvserver/batcheval/testutils_test.go @@ -0,0 +1,17 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package batcheval_test + +import "github.com/cockroachdb/cockroach/pkg/testutils/storageutils" + +type kvs = storageutils.KVs + +var pointKV = storageutils.PointKV diff --git a/pkg/kv/kvserver/replica_rankings_test.go b/pkg/kv/kvserver/replica_rankings_test.go index 632b9031fd73..acae5f42fc0e 100644 --- a/pkg/kv/kvserver/replica_rankings_test.go +++ b/pkg/kv/kvserver/replica_rankings_test.go @@ -19,10 +19,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/sstutil" + "github.com/cockroachdb/cockroach/pkg/testutils/storageutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/stretchr/testify/require" @@ -103,22 +104,18 @@ func TestAddSSTQPSStat(t *testing.T) { nextKey := scratchKey.Next() // Construct an sst with 200 keys that will be reused with different divisors. - sstKeys := make([]sstutil.KV, 200) + sstKeys := make(storageutils.KVs, 200) for i := range sstKeys { - sstKeys[i] = sstutil.KV{ - KeyString: nextKey.String(), - WallTimestamp: 1, - ValueString: "value", - } + sstKeys[i] = storageutils.PointKV(nextKey.String(), 1, "value") nextKey = nextKey.Next() } - sst, start, end := sstutil.MakeSST(t, ts.ClusterSettings(), sstKeys) + sst, start, end := storageutils.MakeSST(t, ts.ClusterSettings(), sstKeys) requestSize := float64(len(sst)) sstReq := &roachpb.AddSSTableRequest{ RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, Data: sst, - MVCCStats: sstutil.ComputeStats(t, sst), + MVCCStats: storageutils.SSTStats(t, sst, 0), } get := &roachpb.GetRequest{ @@ -340,21 +337,17 @@ func TestReadLoadMetricAccounting(t *testing.T) { nextKey := scratchKey.Next() scratchKeys := make([]roachpb.Key, 300) - sstKeys := make([]sstutil.KV, 300) + sstKeys := make(storageutils.KVs, 300) for i := range sstKeys { scratchKeys[i] = nextKey - sstKeys[i] = sstutil.KV{ - KeyString: nextKey.String(), - WallTimestamp: 1, - ValueString: "value", - } + sstKeys[i] = storageutils.PointKV(nextKey.String(), 1, "value") nextKey = nextKey.Next() } - sst, start, end := sstutil.MakeSST(t, ts.ClusterSettings(), sstKeys) + sst, start, end := storageutils.MakeSST(t, ts.ClusterSettings(), sstKeys) sstReq := &roachpb.AddSSTableRequest{ RequestHeader: roachpb.RequestHeader{Key: start, EndKey: end}, Data: sst, - MVCCStats: sstutil.ComputeStats(t, sst), + MVCCStats: storageutils.SSTStats(t, sst, 0), } addSSTBA := roachpb.BatchRequest{} @@ -387,9 +380,9 @@ func TestReadLoadMetricAccounting(t *testing.T) { expectedRBPS float64 }{ {getReadBA, 1, 0, 1, 0, 10}, - {genVariableRead(ctx, start, sstKeys[1].Key()), 1, 0, 1, 0, 38}, - {genVariableRead(ctx, start, sstKeys[4].Key()), 1, 0, 4, 0, 176}, - {genVariableRead(ctx, start, sstKeys[64].Key()), 1, 0, 64, 0, 10496}, + {genVariableRead(ctx, start, sstKeys[1].(storage.MVCCKeyValue).Key.Key), 1, 0, 1, 0, 38}, + {genVariableRead(ctx, start, sstKeys[4].(storage.MVCCKeyValue).Key.Key), 1, 0, 4, 0, 176}, + {genVariableRead(ctx, start, sstKeys[64].(storage.MVCCKeyValue).Key.Key), 1, 0, 64, 0, 10496}, } store, err := ts.GetStores().(*Stores).GetStore(ts.GetFirstStoreID()) diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index 0a3fb68f01e3..eff30f253807 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -2290,6 +2290,10 @@ func TestUnindexedBatchClearAllRangeKeys(t *testing.T) { require.Empty(t, scanRangeKeys(t, eng)) } +// TODO(erikgrinaker): The below test helpers should be moved to +// testutils/storageutils instead, but that requires storage tests to be in the +// storage_test package to avoid import cycles. + // scanRangeKeys scans all range keys from the reader. func scanRangeKeys(t *testing.T, r Reader) []MVCCRangeKeyValue { t.Helper() diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index 8bac4ddbf4b8..c24d2b22295d 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -573,6 +573,8 @@ func TestHasRangeKeyBetween(t *testing.T) { } } +// TODO(erikgrinaker): The below should use the testutils/storageutils variants +// instead, but that requires test code to be in storage_test. func pointKey(key string, ts int) MVCCKey { return MVCCKey{Key: roachpb.Key(key), Timestamp: wallTS(ts)} } diff --git a/pkg/storage/mvcc_value_test.go b/pkg/storage/mvcc_value_test.go index 0066572d83a0..3a1e9ed785b1 100644 --- a/pkg/storage/mvcc_value_test.go +++ b/pkg/storage/mvcc_value_test.go @@ -234,6 +234,8 @@ func BenchmarkDecodeMVCCValue(b *testing.B) { } } +// TODO(erikgrinaker): Use testutils/storageutils instead when test code is +// moved to storage_test to avoid circular deps. func stringValue(s string) MVCCValue { return MVCCValue{Value: roachpb.MakeValueFromString(s)} } diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index f1bd5882b4e3..7c82a2f2be29 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -2148,9 +2148,6 @@ func TestLint(t *testing.T) { `pkg/testutils/.*\.go`, `pkg/workload/.*\.go`, }, "|") + `)` - unkeyedLiteralExceptions := `pkg/.*_test\.go:.*(` + strings.Join([]string{ - `pkg/testutils/sstutil\.KV`, - }, "|") + `)` filters := []stream.Filter{ // Ignore generated files. stream.GrepNot(`pkg/.*\.pb\.go:`), @@ -2216,10 +2213,6 @@ func TestLint(t *testing.T) { // pooling, etc, then test code needs to adhere as well. stream.GrepNot(nakedGoroutineExceptions + `:.*Use of go keyword not allowed`), stream.GrepNot(nakedGoroutineExceptions + `:.*Illegal call to Group\.Go\(\)`), - // We allow unkeyed struct literals for certain internal test types. - // Ideally, go vet should not complain about this for types declared in - // the same module: https://github.com/golang/go/issues/43864 - stream.GrepNot(unkeyedLiteralExceptions + `.*composite literal uses unkeyed fields`), } const vetTool = "roachvet" diff --git a/pkg/testutils/sstutil/BUILD.bazel b/pkg/testutils/sstutil/BUILD.bazel deleted file mode 100644 index 016a51c0e079..000000000000 --- a/pkg/testutils/sstutil/BUILD.bazel +++ /dev/null @@ -1,21 +0,0 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") - -go_library( - name = "sstutil", - srcs = [ - "kv.go", - "sstutil.go", - ], - importpath = "github.com/cockroachdb/cockroach/pkg/testutils/sstutil", - visibility = ["//visibility:public"], - deps = [ - "//pkg/keys", - "//pkg/roachpb", - "//pkg/settings/cluster", - "//pkg/storage", - "//pkg/storage/enginepb", - "//pkg/util/hlc", - "//pkg/util/protoutil", - "@com_github_stretchr_testify//require", - ], -) diff --git a/pkg/testutils/sstutil/kv.go b/pkg/testutils/sstutil/kv.go deleted file mode 100644 index ff2b8ecf16b6..000000000000 --- a/pkg/testutils/sstutil/kv.go +++ /dev/null @@ -1,57 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package sstutil - -import ( - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" -) - -// KV is a simplified representation of an MVCC key/value pair. -type KV struct { - KeyString string - WallTimestamp int64 // 0 for inline - ValueString string // "" for nil (tombstone) -} - -// Key returns the roachpb.Key representation of the key. -func (kv KV) Key() roachpb.Key { - return roachpb.Key(kv.KeyString) -} - -// Timestamp returns the hlc.Timestamp representation of the timestamp. -func (kv KV) Timestamp() hlc.Timestamp { - return hlc.Timestamp{WallTime: kv.WallTimestamp} -} - -// MVCCKey returns the storage.MVCCKey representation of the key and timestamp. -func (kv KV) MVCCKey() storage.MVCCKey { - return storage.MVCCKey{ - Key: kv.Key(), - Timestamp: kv.Timestamp(), - } -} - -// Value returns the roachpb.Value representation of the value. -func (kv KV) Value() roachpb.Value { - value := roachpb.MakeValueFromString(kv.ValueString) - if kv.ValueString == "" { - value = roachpb.Value{} - } - value.InitChecksum(kv.Key()) - return value -} - -// MVCCValue returns the storage.MVCCValue representation of the value. -func (kv KV) MVCCValue() storage.MVCCValue { - return storage.MVCCValue{Value: kv.Value()} -} diff --git a/pkg/testutils/sstutil/sstutil.go b/pkg/testutils/sstutil/sstutil.go deleted file mode 100644 index a0e9cb202647..000000000000 --- a/pkg/testutils/sstutil/sstutil.go +++ /dev/null @@ -1,101 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package sstutil - -import ( - "context" - "testing" - - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" - "github.com/stretchr/testify/require" -) - -// MakeSST builds a binary in-memory SST from the given tests data. It returns -// the binary SST data as well as the start and end (exclusive) keys of the SST. -func MakeSST(t *testing.T, st *cluster.Settings, kvs []KV) ([]byte, roachpb.Key, roachpb.Key) { - t.Helper() - - sstFile := &storage.MemFile{} - writer := storage.MakeIngestionSSTWriter(context.Background(), st, sstFile) - defer writer.Close() - - start, end := keys.MaxKey, keys.MinKey - for _, kv := range kvs { - if kv.Key().Compare(start) < 0 { - start = kv.Key() - } - if kv.Key().Compare(end) > 0 { - end = kv.Key() - } - if kv.Timestamp().IsEmpty() { - meta := &enginepb.MVCCMetadata{RawBytes: kv.Value().RawBytes} - metaBytes, err := protoutil.Marshal(meta) - require.NoError(t, err) - require.NoError(t, writer.PutUnversioned(kv.Key(), metaBytes)) - } else { - require.NoError(t, writer.PutMVCC(kv.MVCCKey(), kv.MVCCValue())) - } - } - require.NoError(t, writer.Finish()) - writer.Close() - - return sstFile.Data(), start, end.Next() -} - -// ScanSST scans a binary in-memory SST for KV pairs. -func ScanSST(t *testing.T, sst []byte) []KV { - t.Helper() - - var kvs []KV - iter, err := storage.NewMemSSTIterator(sst, true) - require.NoError(t, err) - defer iter.Close() - - iter.SeekGE(storage.MVCCKey{Key: keys.MinKey}) - for { - ok, err := iter.Valid() - require.NoError(t, err) - if !ok { - break - } - - k := iter.UnsafeKey() - v, err := storage.DecodeMVCCValue(iter.UnsafeValue()) - require.NoError(t, err) - value, err := v.Value.GetBytes() - require.NoError(t, err) - kvs = append(kvs, KV{ - KeyString: string(k.Key), - WallTimestamp: k.Timestamp.WallTime, - ValueString: string(value), - }) - iter.Next() - } - return kvs -} - -// ComputeStats computes the MVCC stats for the given binary SST. -func ComputeStats(t *testing.T, sst []byte) *enginepb.MVCCStats { - t.Helper() - - iter, err := storage.NewMemSSTIterator(sst, true) - require.NoError(t, err) - defer iter.Close() - - stats, err := storage.ComputeStatsForRange(iter, keys.MinKey, keys.MaxKey, 0) - require.NoError(t, err) - return &stats -} diff --git a/pkg/testutils/storageutils/BUILD.bazel b/pkg/testutils/storageutils/BUILD.bazel index f2972731bcae..9049660ea20c 100644 --- a/pkg/testutils/storageutils/BUILD.bazel +++ b/pkg/testutils/storageutils/BUILD.bazel @@ -2,13 +2,26 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "storageutils", - srcs = ["mocking.go"], + srcs = [ + "kv.go", + "mocking.go", + "scan.go", + "sst.go", + "stats.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/testutils/storageutils", visibility = ["//visibility:public"], deps = [ + "//pkg/keys", "//pkg/kv/kvserver/kvserverbase", "//pkg/roachpb", + "//pkg/settings/cluster", + "//pkg/storage", + "//pkg/storage/enginepb", + "//pkg/util/hlc", + "//pkg/util/protoutil", "//pkg/util/syncutil", "//pkg/util/syncutil/singleflight", + "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/testutils/storageutils/kv.go b/pkg/testutils/storageutils/kv.go new file mode 100644 index 000000000000..a09abe4ca722 --- /dev/null +++ b/pkg/testutils/storageutils/kv.go @@ -0,0 +1,103 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storageutils + +import ( + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/hlc" +) + +// KVs is a slice of either MVCCKeyValue or MVCCRangeKeyValue. +type KVs []interface{} + +// MVCCKeyValues converts the slice to []MVCCKeyValue. +func (kvs KVs) MVCCKeyValues() []storage.MVCCKeyValue { + mvccKVs := []storage.MVCCKeyValue{} + for _, kv := range kvs { + mvccKVs = append(mvccKVs, kv.(storage.MVCCKeyValue)) + } + return mvccKVs +} + +// PointKey creates an MVCCKey for the given string key and timestamp (walltime +// seconds). +func PointKey(key string, ts int) storage.MVCCKey { + return storage.MVCCKey{Key: roachpb.Key(key), Timestamp: WallTS(ts)} +} + +// PointKV creates an MVCCKeyValue for the given string key/value and timestamp +// (walltime seconds). An empty string is a tombstone. +func PointKV(key string, ts int, value string) storage.MVCCKeyValue { + var mvccValue storage.MVCCValue + if value != "" { + mvccValue = StringValue(value) + } + v, err := storage.EncodeMVCCValue(mvccValue) + if err != nil { + panic(err) + } + return storage.MVCCKeyValue{ + Key: PointKey(key, ts), + Value: v, + } +} + +// RangeKey creates an MVCCRangeKey for the given string key and timestamp +// (in walltime seconds). +func RangeKey(start, end string, ts int) storage.MVCCRangeKey { + return storage.MVCCRangeKey{ + StartKey: roachpb.Key(start), + EndKey: roachpb.Key(end), + Timestamp: WallTS(ts), + } +} + +// RangeKV creates an MVCCRangeKeyValue for the given string keys, value, and +// timestamp (in walltime seconds). +func RangeKV(start, end string, ts int, value storage.MVCCValue) storage.MVCCRangeKeyValue { + valueBytes, err := storage.EncodeMVCCValue(value) + if err != nil { + panic(err) + } + if valueBytes == nil { + valueBytes = []byte{} + } + return storage.MVCCRangeKeyValue{ + RangeKey: RangeKey(start, end, ts), + Value: valueBytes, + } +} + +// WallTS creates a timestamp for the given wall time (in seconds). +func WallTS(ts int) hlc.Timestamp { + return hlc.Timestamp{WallTime: int64(ts)} +} + +// StringValue creates an MVCCValue for a string +func StringValue(s string) storage.MVCCValue { + return storage.MVCCValue{Value: roachpb.MakeValueFromString(s)} +} + +// StringValueRaw creates an encoded MVCCValue for a string. +func StringValueRaw(s string) []byte { + b, err := storage.EncodeMVCCValue(StringValue(s)) + if err != nil { + panic(err) + } + return b +} + +// WithLocalTS attaches a local timestamp (in walltime seconds) to an MVCCValue. +func WithLocalTS(v storage.MVCCValue, ts int) storage.MVCCValue { + v.MVCCValueHeader.LocalTimestamp = hlc.ClockTimestamp{WallTime: int64(ts)} + return v +} diff --git a/pkg/testutils/storageutils/scan.go b/pkg/testutils/storageutils/scan.go new file mode 100644 index 000000000000..abaa6d66bb0d --- /dev/null +++ b/pkg/testutils/storageutils/scan.go @@ -0,0 +1,94 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storageutils + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/stretchr/testify/require" +) + +// ScanEngine scans all point/range keys from the replicated user keyspace of +// the engine, returning a combined slice of eyValue and MVCCKeyValue in order. +func ScanEngine(t *testing.T, engine storage.Reader) KVs { + t.Helper() + + iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, + LowerBound: keys.LocalMax, + UpperBound: keys.MaxKey, + }) + defer iter.Close() + return ScanIter(t, iter) +} + +// ScanIter scans all point/range keys from the iterator, and returns a combined +// slice of MVCCRangeKeyValue and MVCCKeyValue in order. +func ScanIter(t *testing.T, iter storage.SimpleMVCCIterator) KVs { + t.Helper() + + var kvs []interface{} + var prevRangeStart roachpb.Key + for iter.SeekGE(storage.MVCCKey{Key: keys.LocalMax}); ; iter.Next() { + ok, err := iter.Valid() + require.NoError(t, err) + if !ok { + break + } + hasPoint, hasRange := iter.HasPointAndRange() + if hasRange { + if bounds := iter.RangeBounds(); !bounds.Key.Equal(prevRangeStart) { + for _, rkv := range iter.RangeKeys() { + kvs = append(kvs, rkv.Clone()) + } + prevRangeStart = bounds.Key.Clone() + } + } + if hasPoint { + var value []byte + if iter.UnsafeKey().IsValue() { + if v := iter.UnsafeValue(); len(v) > 0 { + value = append(value, iter.UnsafeValue()...) + } + } else { + var meta enginepb.MVCCMetadata + require.NoError(t, protoutil.Unmarshal(iter.UnsafeValue(), &meta)) + if meta.RawBytes == nil { + // Skip intent metadata records; we're only interested in the provisional value. + continue + } + value = meta.RawBytes + } + + kvs = append(kvs, storage.MVCCKeyValue{ + Key: iter.UnsafeKey().Clone(), + Value: value, + }) + } + } + return kvs +} + +// ScanSST scans all point/range keys from the given binary SST, returning a +// combined slice of eyValue and MVCCKeyValue in order. +func ScanSST(t *testing.T, sst []byte) KVs { + t.Helper() + + iter, err := storage.NewPebbleMemSSTIterator(sst, true /* verify */) + require.NoError(t, err) + defer iter.Close() + return ScanIter(t, iter) +} diff --git a/pkg/testutils/storageutils/sst.go b/pkg/testutils/storageutils/sst.go new file mode 100644 index 000000000000..1ac7cd988c1f --- /dev/null +++ b/pkg/testutils/storageutils/sst.go @@ -0,0 +1,73 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storageutils + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/stretchr/testify/require" +) + +// MakeSST builds a binary in-memory SST from the given KVs, which can be both +// MVCCKeyValue or MVCCRangeKeyValue. It returns the binary SST data as well as +// the start and end (exclusive) keys of the SST. +func MakeSST( + t *testing.T, st *cluster.Settings, kvs []interface{}, +) ([]byte, roachpb.Key, roachpb.Key) { + t.Helper() + + sstFile := &storage.MemFile{} + writer := storage.MakeIngestionSSTWriter(context.Background(), st, sstFile) + defer writer.Close() + + start, end := keys.MaxKey, keys.MinKey + for _, kvI := range kvs { + var s, e roachpb.Key + switch kv := kvI.(type) { + case storage.MVCCKeyValue: + if kv.Key.Timestamp.IsEmpty() { + v, err := protoutil.Marshal(&enginepb.MVCCMetadata{RawBytes: kv.Value}) + require.NoError(t, err) + require.NoError(t, writer.PutUnversioned(kv.Key.Key, v)) + } else { + require.NoError(t, writer.PutRawMVCC(kv.Key, kv.Value)) + } + s, e = kv.Key.Key, kv.Key.Key.Next() + + case storage.MVCCRangeKeyValue: + v, err := storage.DecodeMVCCValue(kv.Value) + require.NoError(t, err) + require.NoError(t, writer.ExperimentalPutMVCCRangeKey(kv.RangeKey, v)) + s, e = kv.RangeKey.StartKey, kv.RangeKey.EndKey + + default: + t.Fatalf("invalid KV type %T", kv) + } + if s.Compare(start) < 0 { + start = s + } + if e.Compare(end) > 0 { + end = e + } + } + + require.NoError(t, writer.Finish()) + writer.Close() + + return sstFile.Data(), start, end +} diff --git a/pkg/testutils/storageutils/stats.go b/pkg/testutils/storageutils/stats.go new file mode 100644 index 000000000000..f4fd48efb9d3 --- /dev/null +++ b/pkg/testutils/storageutils/stats.go @@ -0,0 +1,47 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storageutils + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/stretchr/testify/require" +) + +// EngineStats computes MVCC stats for the given engine reader, limited to the +// replicated user keyspace. +func EngineStats(t *testing.T, engine storage.Reader, nowNanos int64) *enginepb.MVCCStats { + t.Helper() + + iter := engine.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ + LowerBound: keys.LocalMax, + UpperBound: keys.MaxKey, + }) + defer iter.Close() + stats, err := storage.ComputeStatsForRange(iter, keys.LocalMax, keys.MaxKey, nowNanos) + require.NoError(t, err) + return &stats +} + +// SSTStats computes MVCC stats for the given binary SST. +func SSTStats(t *testing.T, sst []byte, nowNanos int64) *enginepb.MVCCStats { + t.Helper() + + iter, err := storage.NewMemSSTIterator(sst, true) + require.NoError(t, err) + defer iter.Close() + stats, err := storage.ComputeStatsForRange(iter, keys.MinKey, keys.MaxKey, nowNanos) + require.NoError(t, err) + return &stats +}