From ec6e50dda6563a0431d96f9f0fed871a73e5c721 Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Mon, 17 Jul 2023 10:31:18 +0100 Subject: [PATCH 1/7] backupccl: correctly classify tests that start tenants Additionally, TestAllSystemTablesHaveBackupConfig now starts a tenant explicitly to ensure we always test the secondary tenant and system tenant case. Epic: none Release note: None --- pkg/ccl/backupccl/backup_tenant_test.go | 4 +- pkg/ccl/backupccl/backup_test.go | 47 +++++-------------- pkg/ccl/backupccl/system_schema_test.go | 33 +++++++++---- .../backupccl/tenant_backup_nemesis_test.go | 4 +- 4 files changed, 39 insertions(+), 49 deletions(-) diff --git a/pkg/ccl/backupccl/backup_tenant_test.go b/pkg/ccl/backupccl/backup_tenant_test.go index d773c8d6f9f8..5eb3e755575e 100644 --- a/pkg/ccl/backupccl/backup_tenant_test.go +++ b/pkg/ccl/backupccl/backup_tenant_test.go @@ -41,9 +41,7 @@ func TestBackupTenantImportingTable(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - // Test is designed to run with explicit tenants. No need to - // implicitly create a tenant. - DefaultTestTenant: base.TODOTestTenantDisabled, + DefaultTestTenant: base.TestControlsTenantsExplicitly, }, }) defer tc.Stopper().Stop(ctx) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 1f1b39ae1a06..7b24dd8d4415 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -6035,9 +6035,7 @@ func TestProtectedTimestampsDuringBackup(t *testing.T) { params := base.TestClusterArgs{} params.ServerArgs.ExternalIODir = dir params.ServerArgs.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals() - // This test instantiates its own secondary tenants below. No need to run - // it probabilistically under a test tenant. - params.ServerArgs.DefaultTestTenant = base.TODOTestTenantDisabled + params.ServerArgs.DefaultTestTenant = base.TestControlsTenantsExplicitly tc := testcluster.StartTestCluster(t, 1, params) defer tc.Stopper().Stop(ctx) @@ -6455,10 +6453,8 @@ func TestPaginatedBackupTenant(t *testing.T) { const numAccounts = 1 serverArgs := base.TestServerArgs{ - Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, - // Disabled to probabilistically spin up a tenant in each server because the - // test explicitly sets up tenants to test on. - DefaultTestTenant: base.TODOTestTenantDisabled} + Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, + DefaultTestTenant: base.TestControlsTenantsExplicitly} params := base.TestClusterArgs{ServerArgs: serverArgs} var numExportRequests int @@ -6940,9 +6936,7 @@ func TestBackupRestoreTenant(t *testing.T) { }, }, - // Disabled to probabilistically spin up a tenant in each server because the - // test explicitly sets up tenants to test on. - DefaultTestTenant: base.TODOTestTenantDisabled}, + DefaultTestTenant: base.TestControlsTenantsExplicitly}, } const numAccounts = 1 @@ -7027,12 +7021,8 @@ func TestBackupRestoreTenant(t *testing.T) { t.Run("restore-tenant10-to-latest", func(t *testing.T) { restoreTC := testcluster.StartTestCluster( t, singleNode, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ - ExternalIODir: dir, - // This test already exercises the tenant codepaths explicitly - // by creating a tenant. Furthermore, the test requires that - // it run from the system tenant because it restores tenants. - // Disable the default test tenant because it's not necessary. - DefaultTestTenant: base.TODOTestTenantDisabled, + ExternalIODir: dir, + DefaultTestTenant: base.TestControlsTenantsExplicitly, Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, }}, ) @@ -7244,13 +7234,8 @@ func TestBackupRestoreTenant(t *testing.T) { t.Run("restore-all-from-cluster-backup", func(t *testing.T) { restoreTC := testcluster.StartTestCluster( t, singleNode, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ - ExternalIODir: dir, - // This test already exercises the tenant codepaths explicitly - // by creating a tenant. Furthermore, the test requires that - // it run from the system tenant because it queries the - // system.tenants table. Disable the default test tenant because - // it's not necessary. - DefaultTestTenant: base.TODOTestTenantDisabled, + ExternalIODir: dir, + DefaultTestTenant: base.TestControlsTenantsExplicitly, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), TenantTestingKnobs: &sql.TenantTestingKnobs{ @@ -7354,12 +7339,8 @@ func TestBackupRestoreTenant(t *testing.T) { t.Run("restore-tenant10-to-ts1", func(t *testing.T) { restoreTC := testcluster.StartTestCluster( t, singleNode, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ - ExternalIODir: dir, - // This test already exercises the tenant codepaths explicitly - // by creating a tenant. Furthermore, the test requires that - // it run from the system tenant because it restores tenants. - // Disable the default test tenant because it's not necessary. - DefaultTestTenant: base.TODOTestTenantDisabled, + ExternalIODir: dir, + DefaultTestTenant: base.TestControlsTenantsExplicitly, }}, ) defer restoreTC.Stopper().Stop(ctx) @@ -7379,12 +7360,8 @@ func TestBackupRestoreTenant(t *testing.T) { t.Run("restore-tenant20-to-latest", func(t *testing.T) { restoreTC := testcluster.StartTestCluster( t, singleNode, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ - ExternalIODir: dir, - // This test already exercises the tenant codepaths explicitly - // by creating a tenant. Furthermore, the test requires that - // it run from the system tenant because it restores tenants. - // Disable the default test tenant because it's not necessary. - DefaultTestTenant: base.TODOTestTenantDisabled, + ExternalIODir: dir, + DefaultTestTenant: base.TestControlsTenantsExplicitly, }}, ) defer restoreTC.Stopper().Stop(ctx) diff --git a/pkg/ccl/backupccl/system_schema_test.go b/pkg/ccl/backupccl/system_schema_test.go index 0dc8d9b85d32..3dd21579db35 100644 --- a/pkg/ccl/backupccl/system_schema_test.go +++ b/pkg/ccl/backupccl/system_schema_test.go @@ -13,6 +13,9 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -29,20 +32,32 @@ func TestAllSystemTablesHaveBackupConfig(t *testing.T) { tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - // Disabling the test tenant due to test failures. More - // investigation is required. Tracked with #76378. - DefaultTestTenant: base.TODOTestTenantDisabled, + DefaultTestTenant: base.TestControlsTenantsExplicitly, }}) defer tc.Stopper().Stop(ctx) - sqlDB := sqlutils.MakeSQLRunner(tc.Conns[0]) + systemSQL := sqlutils.MakeSQLRunner(tc.Conns[0]) - systemTableNames := sqlDB.QueryStr(t, `USE system; SELECT table_name FROM [SHOW TABLES];`) - for _, systemTableNameRow := range systemTableNames { - systemTableName := systemTableNameRow[0] - if systemTableBackupConfiguration[systemTableName].shouldIncludeInClusterBackup == invalidBackupInclusion { - t.Fatalf("cluster backup inclusion not specified for system table %s", systemTableName) + _, tSQL := serverutils.StartTenant(t, tc.Server(0), base.TestTenantArgs{ + TenantID: roachpb.MustMakeTenantID(10), + TestingKnobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}, + }) + defer tSQL.Close() + + secondaryTenantSQL := sqlutils.MakeSQLRunner(tSQL) + + verifySystemTables := func(tableNames [][]string) { + for _, systemTableNameRow := range tableNames { + systemTableName := systemTableNameRow[0] + if systemTableBackupConfiguration[systemTableName].shouldIncludeInClusterBackup == invalidBackupInclusion { + t.Fatalf("cluster backup inclusion not specified for system table %s", systemTableName) + } } } + tableNamesQuery := `USE system; SELECT table_name FROM [SHOW TABLES];` + + verifySystemTables(systemSQL.QueryStr(t, tableNamesQuery)) + verifySystemTables(secondaryTenantSQL.QueryStr(t, tableNamesQuery)) + } func TestConfigurationDetailsOnlySetForIncludedTables(t *testing.T) { diff --git a/pkg/ccl/backupccl/tenant_backup_nemesis_test.go b/pkg/ccl/backupccl/tenant_backup_nemesis_test.go index 7f59d7ef88db..e73b28de6499 100644 --- a/pkg/ccl/backupccl/tenant_backup_nemesis_test.go +++ b/pkg/ccl/backupccl/tenant_backup_nemesis_test.go @@ -62,7 +62,7 @@ func TestTenantBackupWithCanceledImport(t *testing.T) { tc, hostSQLDB, hostClusterCleanupFn := backupRestoreTestSetupEmpty( t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DefaultTestTenant: base.TODOTestTenantDisabled, + DefaultTestTenant: base.TestControlsTenantsExplicitly, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, @@ -137,7 +137,7 @@ func TestTenantBackupNemesis(t *testing.T) { tc, hostSQLDB, hostClusterCleanupFn := backupRestoreTestSetupEmpty( t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DefaultTestTenant: base.TODOTestTenantDisabled, + DefaultTestTenant: base.TestControlsTenantsExplicitly, Knobs: base.TestingKnobs{ JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, From 89a61668a42692abf75a600e5a9b28301d228ba8 Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Mon, 17 Jul 2023 11:32:46 +0100 Subject: [PATCH 2/7] backupccl: enable tenant testing for TestAlterBackupScheduleEmitsSummary This test works now that we allow nodelocal access for test tenants. Epic: none Release note: None --- pkg/ccl/backupccl/alter_backup_schedule_test.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/pkg/ccl/backupccl/alter_backup_schedule_test.go b/pkg/ccl/backupccl/alter_backup_schedule_test.go index cb5076324307..5621c5ac80dc 100644 --- a/pkg/ccl/backupccl/alter_backup_schedule_test.go +++ b/pkg/ccl/backupccl/alter_backup_schedule_test.go @@ -69,9 +69,6 @@ func newAlterSchedulesTestHelper(t *testing.T) (*alterSchedulesTestHelper, func( args := base.TestServerArgs{ ExternalIODir: dir, - // Some scheduled backup tests fail when run within a tenant. More - // investigation is required. Tracked with #76378. - DefaultTestTenant: base.TODOTestTenantDisabled, Knobs: base.TestingKnobs{ JobsTestingKnobs: knobs, }, From 5f8e35d16e875af6816a13401caa73a54d6c602c Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Mon, 17 Jul 2023 11:33:55 +0100 Subject: [PATCH 3/7] backupccl: enable tenant testing for TestBackupRestoreJobTagAndLabel One assertion called by this test uses the system tenant's SQL server. However, the `UseDatabase` server arg applies to both the tenant and system DB connections -- even if the test only creates the relevant database in one of the tenants. Here, we set the database on the system tenant connection to defaultdb before making queries. Epic: none Release note: None --- pkg/ccl/backupccl/backup_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 7b24dd8d4415..b562a075494d 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -247,7 +247,6 @@ func TestBackupRestoreJobTagAndLabel(t *testing.T) { tc, _, _, cleanupFn := backupRestoreTestSetupWithParams(t, numNodes, numAccounts, InitManualReplication, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ - DefaultTestTenant: base.TODOTestTenantDisabled, Knobs: base.TestingKnobs{ DistSQL: &execinfra.TestingKnobs{ SetupFlowCb: func(ctx context.Context, _ base.SQLInstanceID, _ *execinfrapb.SetupFlowRequest) error { @@ -1004,6 +1003,7 @@ func backupAndRestore( sqlDB := sqlutils.MakeSQLRunner(conn) storageConn := tc.StorageClusterConn() storageSQLDB := sqlutils.MakeSQLRunner(storageConn) + storageSQLDB.Exec(t, "SET DATABASE=defaultdb") { sqlDB.Exec(t, `CREATE INDEX balance_idx ON data.bank (balance)`) testutils.SucceedsSoon(t, func() error { From 5f98741a9d92ffc2df735b1f53af431f6527b0d4 Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Mon, 17 Jul 2023 11:36:58 +0100 Subject: [PATCH 4/7] backupccl: enable tenant testing for TestBackupRestoreSequenceOwnership This test was referencing the wrong kv.DB and codec. Fixing that up allows the test to work without modifying any assertions. Epic: None Release note: None --- pkg/ccl/backupccl/backup_test.go | 49 ++++++++++++++++---------------- 1 file changed, 24 insertions(+), 25 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index b562a075494d..529f35d9c845 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -5428,9 +5428,7 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { const numAccounts = 1 _, origDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) defer cleanupFn() - // Test fails when run within a tenant. More investigation is - // required. Tracked with #76378. - args := base.TestServerArgs{ExternalIODir: dir, DefaultTestTenant: base.TODOTestTenantDisabled} + args := base.TestServerArgs{ExternalIODir: dir} // Setup for sequence ownership backup/restore tests in the same database. backupLoc := localFoo + `/d` @@ -5440,6 +5438,11 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { origDB.Exec(t, `CREATE SEQUENCE d.seq OWNED BY d.t.a`) origDB.Exec(t, `BACKUP DATABASE d TO $1`, backupLoc) + getTableDescriptorFromTestCluster := func(tc *testcluster.TestCluster, database string, table string) catalog.TableDescriptor { + srv := tc.TenantOrServer(0) + return desctestutils.TestingGetPublicTableDescriptor(srv.DB(), srv.Codec(), database, table) + } + // When restoring a database which has a owning table and an owned sequence, // the ownership relationship should be preserved and remapped post restore. t.Run("test restoring database should preserve ownership dependency", func(t *testing.T) { @@ -5447,12 +5450,11 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { defer tc.Stopper().Stop(context.Background()) newDB := sqlutils.MakeSQLRunner(tc.Conns[0]) - kvDB := tc.Server(0).DB() newDB.Exec(t, `RESTORE DATABASE d FROM $1`, backupLoc) - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d", "t") - seqDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d", "seq") + tableDesc := getTableDescriptorFromTestCluster(tc, "d", "t") + seqDesc := getTableDescriptorFromTestCluster(tc, "d", "seq") require.True(t, seqDesc.GetSequenceOpts().HasOwner(), "no sequence owner after restore") require.Equal(t, tableDesc.GetID(), seqDesc.GetSequenceOpts().SequenceOwner.OwnerTableID, @@ -5477,14 +5479,14 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { defer tc.Stopper().Stop(context.Background()) newDB := sqlutils.MakeSQLRunner(tc.Conns[0]) - kvDB := tc.Server(0).DB() + newDB.Exec(t, `CREATE DATABASE d`) newDB.Exec(t, `USE d`) newDB.ExpectErr(t, `pq: cannot restore sequence "seq" without referenced owner`, `RESTORE TABLE seq FROM $1`, backupLoc) newDB.Exec(t, `RESTORE TABLE seq FROM $1 WITH skip_missing_sequence_owners`, backupLoc) - seqDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d", "seq") + seqDesc := getTableDescriptorFromTestCluster(tc, "d", "seq") require.False(t, seqDesc.GetSequenceOpts().HasOwner(), "unexpected owner of restored sequence.") }) @@ -5499,7 +5501,7 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { defer tc.Stopper().Stop(context.Background()) newDB := sqlutils.MakeSQLRunner(tc.Conns[0]) - kvDB := tc.Server(0).DB() + newDB.Exec(t, `CREATE DATABASE d`) newDB.Exec(t, `USE d`) newDB.ExpectErr(t, `pq: cannot restore sequence "seq" without referenced owner table`, @@ -5509,7 +5511,7 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { `RESTORE TABLE t FROM $1`, backupLoc) newDB.Exec(t, `RESTORE TABLE t FROM $1 WITH skip_missing_sequence_owners`, backupLoc) - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d", "t") + tableDesc := getTableDescriptorFromTestCluster(tc, "d", "t") require.Equal(t, 0, tableDesc.PublicColumns()[0].NumOwnsSequences(), "expected restored table to own 0 sequences", @@ -5519,7 +5521,7 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { `RESTORE TABLE seq FROM $1`, backupLoc) newDB.Exec(t, `RESTORE TABLE seq FROM $1 WITH skip_missing_sequence_owners`, backupLoc) - seqDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d", "seq") + seqDesc := getTableDescriptorFromTestCluster(tc, "d", "seq") require.False(t, seqDesc.GetSequenceOpts().HasOwner(), "unexpected sequence owner after restore") }) @@ -5530,13 +5532,12 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { defer tc.Stopper().Stop(context.Background()) newDB := sqlutils.MakeSQLRunner(tc.Conns[0]) - kvDB := tc.Server(0).DB() newDB.Exec(t, `CREATE DATABASE restore_db`) newDB.Exec(t, `RESTORE d.* FROM $1 WITH into_db='restore_db'`, backupLoc) - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "restore_db", "t") - seqDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "restore_db", "seq") + tableDesc := getTableDescriptorFromTestCluster(tc, "restore_db", "t") + seqDesc := getTableDescriptorFromTestCluster(tc, "restore_db", "seq") require.True(t, seqDesc.GetSequenceOpts().HasOwner(), "no sequence owner after restore") require.Equal(t, tableDesc.GetID(), seqDesc.GetSequenceOpts().SequenceOwner.OwnerTableID, @@ -5579,14 +5580,13 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { defer tc.Stopper().Stop(context.Background()) newDB := sqlutils.MakeSQLRunner(tc.Conns[0]) - kvDB := tc.Server(0).DB() newDB.ExpectErr(t, "pq: cannot restore sequence \"seq\" without referenced owner|"+ "pq: cannot restore table \"t\" without referenced sequence", `RESTORE DATABASE d2 FROM $1`, backupLocD2D3) newDB.Exec(t, `RESTORE DATABASE d2 FROM $1 WITH skip_missing_sequence_owners`, backupLocD2D3) - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d2", "t") + tableDesc := getTableDescriptorFromTestCluster(tc, "d2", "t") require.Equal(t, 0, tableDesc.PublicColumns()[0].NumOwnsSequences(), "expected restored table to own no sequences.", ) @@ -5596,12 +5596,12 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { `RESTORE DATABASE d3 FROM $1`, backupLocD2D3) newDB.Exec(t, `RESTORE DATABASE d3 FROM $1 WITH skip_missing_sequence_owners`, backupLocD2D3) - seqDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d3", "seq") + seqDesc := getTableDescriptorFromTestCluster(tc, "d3", "seq") require.False(t, seqDesc.GetSequenceOpts().HasOwner(), "unexpected sequence owner after restore") // Sequence dependencies inside the database should still be preserved. - sd := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d3", "seq2") - td := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d3", "t") + sd := getTableDescriptorFromTestCluster(tc, "d3", "seq2") + td := getTableDescriptorFromTestCluster(tc, "d3", "t") require.True(t, sd.GetSequenceOpts().HasOwner(), "no owner found for seq2") require.Equal(t, td.GetID(), sd.GetSequenceOpts().SequenceOwner.OwnerTableID, @@ -5624,13 +5624,12 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { defer tc.Stopper().Stop(context.Background()) newDB := sqlutils.MakeSQLRunner(tc.Conns[0]) - kvDB := tc.Server(0).DB() newDB.Exec(t, `RESTORE DATABASE d2, d3 FROM $1`, backupLocD2D3) // d2.t owns d3.seq should be preserved. - tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d2", "t") - seqDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d3", "seq") + tableDesc := getTableDescriptorFromTestCluster(tc, "d2", "t") + seqDesc := getTableDescriptorFromTestCluster(tc, "d3", "seq") require.True(t, seqDesc.GetSequenceOpts().HasOwner(), "no sequence owner after restore") require.Equal(t, tableDesc.GetID(), seqDesc.GetSequenceOpts().SequenceOwner.OwnerTableID, @@ -5647,9 +5646,9 @@ func TestBackupRestoreSequenceOwnership(t *testing.T) { ) // d3.t owns d2.seq and d3.seq2 should be preserved. - td := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d3", "t") - sd := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d2", "seq") - sdSeq2 := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "d3", "seq2") + td := getTableDescriptorFromTestCluster(tc, "d3", "t") + sd := getTableDescriptorFromTestCluster(tc, "d2", "seq") + sdSeq2 := getTableDescriptorFromTestCluster(tc, "d3", "seq2") require.True(t, sd.GetSequenceOpts().HasOwner(), "no sequence owner after restore") require.True(t, sdSeq2.GetSequenceOpts().HasOwner(), "no sequence owner after restore") From 99422ec30673aab2716dce5bb02edd3d024b4adf Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Mon, 17 Jul 2023 11:38:07 +0100 Subject: [PATCH 5/7] backupccl: classify tenant TestRestoreErrorPropagates exception Even with the changes I made to the test, this test still does not work when run in a tenant because it depends on disabling span configuration reconciliation to avoid flakiness. Epic: none Release note: None --- pkg/ccl/backupccl/backup_test.go | 30 +++++++++++++++++++++++------- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 529f35d9c845..c92cb31dfd89 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -6342,12 +6342,17 @@ func TestRestoreErrorPropagates(t *testing.T) { defer dirCleanupFn() params := base.TestClusterArgs{} params.ServerArgs.ExternalIODir = dir - // When this test runs under the default test tenant, the RESTORE command - // below which is expected to fail, doesn't. This may be a problem with the - // testing knobs being incorrectly applied to the cluster. More - // investigation is required. Tracked with #76378. - params.ServerArgs.DefaultTestTenant = base.TODOTestTenantDisabled - jobsTableKey := keys.SystemSQLCodec.TablePrefix(uint32(systemschema.JobsTable.GetID())) + // TODO(ssd): The way we inject the error requires that we + // intercept the actual batch request for job table write. To + // keep this from being flakey, we need to disable various + // automatic jobs. Unfortunately, if we disable the span + // configuration job, we can't start a tenant. + params.ServerArgs.DefaultTestTenant = base.TestNeedsTightIntegrationBetweenAPIsAndTestingKnobs + var jobsTableKey = struct { + syncutil.Mutex + key roachpb.Key + }{} + var shouldFail, failures int64 params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ TestingRequestFilter: func(ctx context.Context, ba *kvpb.BatchRequest) *kvpb.Error { @@ -6357,12 +6362,18 @@ func TestRestoreErrorPropagates(t *testing.T) { if !ba.IsWrite() { return nil } + jobsTableKey.Lock() + defer jobsTableKey.Unlock() + if len(jobsTableKey.key) == 0 { + return nil + } + for _, ru := range ba.Requests { r := ru.GetInner() switch r.(type) { case *kvpb.ConditionalPutRequest, *kvpb.PutRequest: key := r.Header().Key - if bytes.HasPrefix(key, jobsTableKey) && atomic.LoadInt64(&shouldFail) > 0 { + if bytes.HasPrefix(key, jobsTableKey.key) && atomic.LoadInt64(&shouldFail) > 0 { return kvpb.NewError(errors.Errorf("boom %d", atomic.AddInt64(&failures, 1))) } } @@ -6378,6 +6389,11 @@ func TestRestoreErrorPropagates(t *testing.T) { defer tc.Stopper().Stop(ctx) db := tc.ServerConn(0) runner := sqlutils.MakeSQLRunner(db) + + jobsTableKey.Lock() + jobsTableKey.key = tc.TenantOrServer(0).Codec().TablePrefix(uint32(systemschema.JobsTable.GetID())) + jobsTableKey.Unlock() + runner.Exec(t, `SET CLUSTER SETTING jobs.metrics.interval.poll = '30s'`) runner.Exec(t, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false`) runner.Exec(t, `SET CLUSTER SETTING sql.stats.system_tables_autostats.enabled = false`) From ddb5cdd1a7204a60e3fe7baf1bd43733e447f845 Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Mon, 17 Jul 2023 11:40:31 +0100 Subject: [PATCH 6/7] backupccl: correctly classify another tenant-controlling test Epic: none Release note: None --- pkg/ccl/backupccl/backup_test.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index c92cb31dfd89..b4e43cc90548 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -7196,12 +7196,8 @@ func TestBackupRestoreTenant(t *testing.T) { t.Run("restore-t10-from-cluster-backup", func(t *testing.T) { restoreTC := testcluster.StartTestCluster( t, singleNode, base.TestClusterArgs{ServerArgs: base.TestServerArgs{ - ExternalIODir: dir, - // This test already exercises the tenant codepaths explicitly - // by creating a tenant. Furthermore, the test requires that - // it run from the system tenant because it restores tenants. - // Disable the default test tenant because it's not necessary. - DefaultTestTenant: base.TODOTestTenantDisabled, + ExternalIODir: dir, + DefaultTestTenant: base.TestControlsTenantsExplicitly, }}, ) defer restoreTC.Stopper().Stop(ctx) From d13fae1a90fd5b2205676a14360117197af64612 Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Mon, 17 Jul 2023 11:41:03 +0100 Subject: [PATCH 7/7] backupccl: enable tenant testing for memory monitoring tests These tests work without modification. Epic: none Release note: None --- pkg/ccl/backupccl/backup_test.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index b4e43cc90548..18f32256fb1d 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -11135,7 +11135,6 @@ func TestRestoreMemoryMonitoringWithShadowing(t *testing.T) { restoreProcessorKnobCount := atomic.Uint32{} args := base.TestServerArgs{ - DefaultTestTenant: base.TODOTestTenantDisabled, Knobs: base.TestingKnobs{ DistSQL: &execinfra.TestingKnobs{ BackupRestoreTestingKnobs: &sql.BackupRestoreTestingKnobs{ @@ -11187,11 +11186,7 @@ func TestRestoreMemoryMonitoringMinWorkerMemory(t *testing.T) { defer log.Scope(t).Close(t) const numAccounts = 100 - args := base.TestServerArgs{ - DefaultTestTenant: base.TODOTestTenantDisabled, - } - params := base.TestClusterArgs{ServerArgs: args} - _, sqlDB, _, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, numAccounts, InitManualReplication, params) + _, sqlDB, _, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, numAccounts, InitManualReplication, base.TestClusterArgs{}) defer cleanupFn() // 4 restore workers means we need minimum 2 workers to start restore.