Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
83389: testutils: add `storageutils` test utilities r=nicktrav a=erikgrinaker

This patch adds a bunch of test utilities to `storageutils`, replacing
the old `sstutil` package. This is done to ease testing of MVCC range
keys in tests outside the `storage` package.

Unfortunately, these are mostly duplicates of utilities in `storage`.
Storage tests use the `storage` package rather than `storage_test`, and
can't make use of `storageutils` yet because it causes an import cycle
with `storage`. This will (hopefully) be addressed separately.

Release note: None

83448: cli: skip TestRemoveDeadReplicas r=erikgrinaker a=tbg

Refs: #75133

Reason: flaky test

Generated by bin/skip-test.

Release justification: non-production code changes

Release note: None

83561: backupccl: deflake TestMetadataSST r=stevendanna a=stevendanna

This is a temporary fix for failures in TestMetadataSST.

First,

    backup_metadata_test.go:49: error executing 'BACKUP TO $1': pq: a
    CCL binary is required to use this statement type: *tree.Backup

is solved by moving the test back into the backupccl package so that
the plan hook for BACKUP is definitely registered.

Second,

    backup_metadata_test.go:89: file /0/BACKUP_MANIFEST does not exist
    in the UserFileTableSystem: external_storage: file doesn't exist

is solved by disabling tenants by default in the setup functions in
backuputils. There is duplication between these functions and the
functions in backupccl, where we already disable tenants. The error
above is likely the result of us directly using the internal executor
of the server to query the userfile storage, which may be incorrect if
we were actually talking to a tenant.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Tobias Grieger <[email protected]>
Co-authored-by: Steven Danna <[email protected]>
  • Loading branch information
4 people committed Jun 29, 2022
4 parents bb1844b + da020b0 + dc9e358 + 1bbc4c2 commit 73e4816
Show file tree
Hide file tree
Showing 24 changed files with 726 additions and 600 deletions.
2 changes: 2 additions & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package backupinfo_test
package backupccl_test

import (
"context"
Expand Down
22 changes: 1 addition & 21 deletions pkg/ccl/backupccl/backupinfo/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
],
)

Expand Down
4 changes: 4 additions & 0 deletions pkg/ccl/backupccl/backuputils/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand All @@ -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
}
}
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/debug_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvclient/rangefeed/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
50 changes: 34 additions & 16 deletions pkg/kv/kvclient/rangefeed/rangefeed_external_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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) {
Expand Down Expand Up @@ -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)
Expand All @@ -530,14 +540,16 @@ 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
// scan if an OnSSTable event is emitted and no OnSSTable event handler is set.
func TestWithOnSSTableCatchesUpIfNotSet(t *testing.T) {
defer leaktest.AfterTest(t)()

storage.DisableMetamorphicSimpleValueEncoding(t)

ctx, cancel := context.WithCancel(context.Background())
defer cancel()

Expand Down Expand Up @@ -587,27 +599,33 @@ 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)
require.Nil(t, pErr)

// 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)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ go_test(
"declare_test.go",
"intent_test.go",
"main_test.go",
"testutils_test.go",
"transaction_test.go",
],
embed = [":batcheval"],
Expand Down Expand Up @@ -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",
Expand Down
Loading

0 comments on commit 73e4816

Please sign in to comment.