Skip to content

Commit

Permalink
backupccl: allow backupRestoreTestSetup[Empty]WithParams to test with…
Browse files Browse the repository at this point in the history
…in tenant

Previously, all backup unit tests that called backupRestoreTestSetupWithParams
and backupRestoreTestSetupEmptyWithParams were disabled to run within tenants,
signficantly decreasing test coverage.

This patch allows tests that call these helper methods to be probabilistically
run within tenants (including most data driven tests). This patch also manually
disables some of these tests to be run within tenants. Future work should
continue to enable bulk unit tests to run within tenants by default.

Fixes #88381, #88527, #88453, #88380
Release note: None
  • Loading branch information
msbutler committed Sep 27, 2022
1 parent 0961abd commit c321fef
Show file tree
Hide file tree
Showing 25 changed files with 224 additions and 116 deletions.
104 changes: 74 additions & 30 deletions pkg/ccl/backupccl/backup_test.go

Large diffs are not rendered by default.

27 changes: 23 additions & 4 deletions pkg/ccl/backupccl/datadriven_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts"
Expand Down Expand Up @@ -127,13 +128,15 @@ type serverCfg struct {
localities string
beforeVersion string
testingKnobCfg string
disableTenant bool
}

func (d *datadrivenTestState) addServer(t *testing.T, cfg serverCfg) error {
var tc serverutils.TestClusterInterface
var cleanup func()
params := base.TestClusterArgs{}
params.ServerArgs.ExternalIODirConfig = cfg.ioConf
params.ServerArgs.DisableDefaultTestTenant = cfg.disableTenant
params.ServerArgs.Knobs = base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
}
Expand Down Expand Up @@ -266,6 +269,9 @@ func (d *datadrivenTestState) getSQLDB(t *testing.T, server string, user string)
//
// - testingKnobCfg: specifies a key to a hardcoded testingKnob configuration
//
// - disable-tenant : ensures the test is never run in a multitenant environment by
// setting testserverargs.DisableDefaultTestTenant to true.
//
// - "upgrade-server version=<version>"
// Upgrade the cluster version of the active server to the passed in
// clusterVersion key. See cockroach_versions.go for possible values.
Expand Down Expand Up @@ -443,6 +449,7 @@ func TestDataDriven(t *testing.T) {
case "new-server":
var name, shareDirWith, iodir, localities, beforeVersion, testingKnobCfg string
var splits int
var disableTenant bool
nodes := singleNode
var io base.ExternalIODirConfig
d.ScanArgs(t, "name", &name)
Expand All @@ -469,10 +476,17 @@ func TestDataDriven(t *testing.T) {
}
if d.HasArg("beforeVersion") {
d.ScanArgs(t, "beforeVersion", &beforeVersion)
if !d.HasArg("disable-tenant") {
// TODO(msbutler): figure out why test tenants don't mix with version testing
t.Fatal("tests that use beforeVersion must use disable-tenant")
}
}
if d.HasArg("testingKnobCfg") {
d.ScanArgs(t, "testingKnobCfg", &testingKnobCfg)
}
if d.HasArg("disable-tenant") {
disableTenant = true
}

lastCreatedServer = name
cfg := serverCfg{
Expand All @@ -484,6 +498,7 @@ func TestDataDriven(t *testing.T) {
localities: localities,
beforeVersion: beforeVersion,
testingKnobCfg: testingKnobCfg,
disableTenant: disableTenant,
}
err := ds.addServer(t, cfg)
if err != nil {
Expand Down Expand Up @@ -747,11 +762,15 @@ func TestDataDriven(t *testing.T) {

case "create-dummy-system-table":
db := ds.servers[lastCreatedServer].DB()
codec := ds.servers[lastCreatedServer].ExecutorConfig().(sql.ExecutorConfig).Codec
execCfg := ds.servers[lastCreatedServer].ExecutorConfig().(sql.ExecutorConfig)
testTenants := ds.servers[lastCreatedServer].TestTenants()
if len(testTenants) > 0 {
execCfg = testTenants[0].ExecutorConfig().(sql.ExecutorConfig)
}
codec := execCfg.Codec
dummyTable := systemschema.SettingsTable
err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
id, err := ds.servers[lastCreatedServer].ExecutorConfig().(sql.ExecutorConfig).
DescIDGenerator.GenerateUniqueDescID(ctx)
id, err := execCfg.DescIDGenerator.GenerateUniqueDescID(ctx)
if err != nil {
return err
}
Expand Down Expand Up @@ -819,7 +838,7 @@ func handleKVRequest(
err := ds.getSQLDB(t, server, user).QueryRow(`SELECT id FROM system.namespace WHERE name = $1`,
target).Scan(&tableID)
require.NoError(t, err)
bankSpan := makeTableSpan(tableID)
bankSpan := makeTableSpan(keys.SystemSQLCodec, tableID)
dr := roachpb.DeleteRangeRequest{
// Bogus span to make it a valid request.
RequestHeader: roachpb.RequestHeader{
Expand Down
22 changes: 21 additions & 1 deletion pkg/ccl/backupccl/full_cluster_backup_restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,12 @@ func TestFullClusterBackup(t *testing.T) {
settings := clustersettings.MakeTestingClusterSettings()
params := base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Settings: settings,
// Disabled only because backupRestoreTestSetupEmpty, another DR test
// helper function, that is not yet enabled to set up tenants within
// clusters by default. Tracking issue
// https://github.com/cockroachdb/cockroach/issues/76378
DisableDefaultTestTenant: true,
Settings: settings,
Knobs: base.TestingKnobs{
SpanConfig: &spanconfig.TestingKnobs{
// We compare job progress before and after a restore. Disable
Expand Down Expand Up @@ -351,6 +356,11 @@ func TestSingletonSpanConfigJobPostRestore(t *testing.T) {

params := base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
// Disabled only because backupRestoreTestSetupEmpty, another DR test
// helper function, is not yet enabled to set up tenants within
// clusters by default. Tracking issue
// https://github.com/cockroachdb/cockroach/issues/76378
DisableDefaultTestTenant: true,
Knobs: base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
},
Expand Down Expand Up @@ -942,6 +952,11 @@ func TestReintroduceOfflineSpans(t *testing.T) {
}},
}
params.ServerArgs.Knobs = knobs
// Disabled only because backupRestoreTestSetupEmpty, another DR test
// helper function, is not yet enabled to set up tenants within
// clusters by default. Tracking issue
// https://github.com/cockroachdb/cockroach/issues/76378
params.ServerArgs.DisableDefaultTestTenant = true

const numAccounts = 1000
ctx := context.Background()
Expand Down Expand Up @@ -1098,6 +1113,11 @@ func TestFullClusterRestoreWithUserIDs(t *testing.T) {

params := base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
// Disabled only because backupRestoreTestSetupEmpty, another DR test
// helper function, that is not yet enabled to set up tenants within
// clusters by default. Tracking issue
// https://github.com/cockroachdb/cockroach/issues/76378
DisableDefaultTestTenant: true,
Knobs: base.TestingKnobs{
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
},
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/restore_span_covering_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ func MockBackupChain(length, spans, baseFiles int, r *rand.Rand) []backuppb.Back
backups[i].IntroducedSpans = make(roachpb.Spans, 0)
for j := range backups[i].Spans {
tableID := genTableID(j)
backups[i].Spans[j] = makeTableSpan(tableID)
backups[i].Spans[j] = makeTableSpan(keys.SystemSQLCodec, tableID)
}
backups[i].EndTime = ts.Add(time.Minute.Nanoseconds()*int64(i), 0)
if i > 0 {
Expand All @@ -66,7 +66,7 @@ func MockBackupChain(length, spans, baseFiles int, r *rand.Rand) []backuppb.Back
// span[spanIdxToDrop], present in the first i backups, and add a new
// one.
newTableID := genTableID(spanIdxToDrop) + 1
backups[i].Spans[spanIdxToDrop] = makeTableSpan(newTableID)
backups[i].Spans[spanIdxToDrop] = makeTableSpan(keys.SystemSQLCodec, newTableID)
backups[i].IntroducedSpans = append(backups[i].IntroducedSpans, backups[i].Spans[spanIdxToDrop])
}

Expand Down
4 changes: 3 additions & 1 deletion pkg/ccl/backupccl/testdata/backup-restore/column-families
Original file line number Diff line number Diff line change
@@ -1,4 +1,6 @@
new-server name=s1 localities=us-east-1,us-west-1,us-west-2,eu-central-1
# disabled to run within tenant because ALTER SPLIT cmd is not supported within tenant

new-server name=s1 disable-tenant localities=us-east-1,us-west-1,us-west-2,eu-central-1
----

exec-sql
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,8 +3,11 @@
# appropriately during a cluster restore. The conflicting system tables in the
# restoring cluster should be copied to a descriptor ID higher than any
# descriptor in the backup.
#
# disabled to run within a tenant because they cannot set zone configs
# https://github.com/cockroachdb/cockroach/issues/49854?version=v22.2

new-server name=s1
new-server name=s1 disable-tenant
----

exec-sql
Expand All @@ -23,7 +26,7 @@ exec-sql
BACKUP INTO 'nodelocal://0/conflicting-descriptors';
----

new-server name=s2 share-io-dir=s1
new-server name=s2 share-io-dir=s1 disable-tenant
----

# Create 4 dummy system tables that will have conflicting IDs with the database,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,16 @@ exec-sql
BACKUP INTO 'external://conn-foo/cluster';
----

# filter out the tenant_settings row to ensure the test can run within and outside a tenant
query-sql
SELECT object_name, object_type, backup_type FROM [SHOW BACKUP LATEST IN
'external://conn-foo/cluster'] ORDER BY object_name;
SELECT
object_name, object_type, backup_type
FROM
[SHOW BACKUP LATEST IN 'external://conn-foo/cluster']
WHERE
object_name != 'tenant_settings'
ORDER BY
object_name;
----
bank table full
comments table full
Expand All @@ -45,7 +52,6 @@ scheduled_jobs table full
schema schema full
settings table full
system database full
tenant_settings table full
ui table full
users table full
zones table full
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,16 @@ exec-sql
BACKUP INTO 'external://conn-foo/cluster';
----

# filter out the tenant_settings row to ensure the test can run within and outside a tenant
query-sql
SELECT object_name, object_type, backup_type FROM [SHOW BACKUP LATEST IN
'external://conn-foo/cluster'] ORDER BY object_name;
SELECT
object_name, object_type, backup_type
FROM
[SHOW BACKUP LATEST IN 'external://conn-foo/cluster']
WHERE
object_name != 'tenant_settings'
ORDER BY
object_name;
----
bank table full
comments table full
Expand All @@ -45,7 +52,6 @@ scheduled_jobs table full
schema schema full
settings table full
system database full
tenant_settings table full
ui table full
userfiles_root_upload_files table full
userfiles_root_upload_payload table full
Expand Down
4 changes: 3 additions & 1 deletion pkg/ccl/backupccl/testdata/backup-restore/feature-flags
Original file line number Diff line number Diff line change
@@ -1,6 +1,8 @@
subtest backup-feature-flags

new-server name=s1
# disabled for tenants as they can't enable/disable backup and restore features

new-server name=s1 disable-tenant
----

exec-sql
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,9 @@
# - roll it back it back non-mvcc
# - run an inc backup and ensure we reintroduce the table spans

new-server name=s1
# disabled to run within tenant as they don't have access to the
# storage.mvcc.range_tombstones.enabled cluster setting
new-server name=s1 disable-tenant
----

###########
Expand Down
7 changes: 5 additions & 2 deletions pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports
Original file line number Diff line number Diff line change
Expand Up @@ -329,9 +329,12 @@ SELECT * FROM d.foofoo;
# RESTORE AOST in-progress IMPORT, these tables should get thrown out.
#
# TODO(msbutler): cover mixed version RESTORE TABLE
#
# Disabled to run within tenants because version gating infra does not play nice within tenants.
# More investigation required.


new-server name=s4 share-io-dir=s1 allow-implicit-access beforeVersion=Start22_2
new-server name=s4 share-io-dir=s1 allow-implicit-access beforeVersion=Start22_2 disable-tenant
----

exec-sql
Expand Down Expand Up @@ -529,7 +532,7 @@ d foofoo table 3 incremental


# Restore the backups taken from a mixed version chain
new-server name=s5 share-io-dir=s1 allow-implicit-access
new-server name=s5 share-io-dir=s1 allow-implicit-access disable-tenant
----


Expand Down
8 changes: 4 additions & 4 deletions pkg/ccl/backupccl/testdata/backup-restore/max-row-size
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,10 @@ exec-sql
SET CLUSTER SETTING sql.guardrails.max_row_size_err = '16KiB';
----

query-sql
exec-sql expect-error-regex=(row larger than max row size: table 109 family 0 primary key .*/Table/109/1/2/0 size .*)
INSERT INTO maxrow VALUES (2, repeat('x', 20000))
----
pq: row larger than max row size: table 109 family 0 primary key /Table/109/1/2/0 size 20013
regex matches error

exec-sql
BACKUP maxrow INTO 'nodelocal://1/maxrow';
Expand All @@ -39,10 +39,10 @@ SELECT i, pg_column_size(s) FROM d2.maxrow ORDER BY i;
----
1 20004

query-sql
exec-sql expect-error-regex=(row larger than max row size: table 112 family 0 primary key .*/Table/112/1/2/0 size .*)
INSERT INTO d2.maxrow VALUES (2, repeat('y', 20000));
----
pq: row larger than max row size: table 112 family 0 primary key /Table/112/1/2/0 size 20014
regex matches error

exec-sql
SET CLUSTER SETTING sql.guardrails.max_row_size_err = DEFAULT;
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,6 @@
new-server name=s1 allow-implicit-access localities=us-east-1,us-west-1
# disabled to run within tenant because multiregion primitives are not supported within tenant

new-server name=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1
----

exec-sql
Expand Down
10 changes: 6 additions & 4 deletions pkg/ccl/backupccl/testdata/backup-restore/multiregion
Original file line number Diff line number Diff line change
@@ -1,4 +1,6 @@
new-server name=s1 allow-implicit-access localities=us-east-1,us-west-1,eu-central-1
# disabled to run within tenant because multiregion primitives are not supported within tenant

new-server name=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1,eu-central-1
----

exec-sql
Expand All @@ -23,7 +25,7 @@ BACKUP INTO 'nodelocal://1/full_cluster_backup/';
----

# A new cluster with the same locality settings.
new-server name=s2 share-io-dir=s1 allow-implicit-access localities=us-east-1,us-west-1,eu-central-1
new-server name=s2 share-io-dir=s1 allow-implicit-access disable-tenant localities=us-east-1,us-west-1,eu-central-1
----

exec-sql
Expand All @@ -48,7 +50,7 @@ postgres root <nil> <nil> {} <nil>
system node <nil> <nil> {} <nil>

# A new cluster with different localities settings.
new-server name=s3 share-io-dir=s1 allow-implicit-access localities=eu-central-1,eu-north-1
new-server name=s3 share-io-dir=s1 allow-implicit-access disable-tenant localities=eu-central-1,eu-north-1
----

exec-sql
Expand Down Expand Up @@ -140,7 +142,7 @@ BACKUP DATABASE eu_central_db INTO 'nodelocal://1/eu_central_database_backup/';
----

# New cluster for a cluster backup.
new-server name=s4 share-io-dir=s1 allow-implicit-access localities=eu-central-1,eu-north-1
new-server name=s4 share-io-dir=s1 allow-implicit-access disable-tenant localities=eu-central-1,eu-north-1
----

exec-sql ignore-notice
Expand Down
4 changes: 3 additions & 1 deletion pkg/ccl/backupccl/testdata/backup-restore/rangekeys
Original file line number Diff line number Diff line change
@@ -1,7 +1,9 @@
# Tests that Backups without Revisions History and Restore properly handle
# range keys

new-server name=s1
# disabled to run within tenants because the kv request cmd only works on system tenants

new-server name=s1 disable-tenant
----

exec-sql
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,10 @@
# - t4: one insert in baz
# - incremental backup

new-server name=s1
# disabled to run within tenants because the kv request cmd only works on system tenants


new-server name=s1 disable-tenant
----

exec-sql
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
new-server name=s1 beforeVersion=Start22_2
new-server name=s1 beforeVersion=Start22_2 disable-tenant
----

exec-sql
Expand All @@ -15,7 +15,7 @@ BACKUP INTO 'nodelocal://1/full_cluster_backup/';
# This is a server where the cluster version is behind the binary version. Such
# a condition only occurs when the user has upgraded the node to a new major
# version but has not yet finalized the upgrade.
new-server name=s2 beforeVersion=Start22_2 share-io-dir=s1
new-server name=s2 beforeVersion=Start22_2 share-io-dir=s1 disable-tenant
----

exec-sql expect-error-regex=pq: cluster restore not supported during major version upgrade: restore started at cluster version 22.1 but binary version is.*
Expand Down
Loading

0 comments on commit c321fef

Please sign in to comment.