From 05aed7c1f7672226d519891016535331bfcaa57b Mon Sep 17 00:00:00 2001 From: adityamaru Date: Wed, 7 Sep 2022 17:04:32 -0400 Subject: [PATCH 1/5] importer: deflake TestImportWorkerFailure Release note: None Fixes: #73546 --- pkg/sql/importer/import_stmt_test.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go index 48a10f9b8da0..3b969be70ab6 100644 --- a/pkg/sql/importer/import_stmt_test.go +++ b/pkg/sql/importer/import_stmt_test.go @@ -5106,7 +5106,6 @@ func TestImportControlJobRBAC(t *testing.T) { // worker node. func TestImportWorkerFailure(t *testing.T) { defer leaktest.AfterTest(t)() - skip.WithIssue(t, 73546, "flaky test") defer log.Scope(t).Close(t) allowResponse := make(chan struct{}) @@ -5122,6 +5121,11 @@ func TestImportWorkerFailure(t *testing.T) { conn := tc.ServerConn(0) sqlDB := sqlutils.MakeSQLRunner(conn) + // Lower the initial buffering adder ingest size to allow import jobs to run + // without exceeding the test memory monitor. + sqlDB.Exec(t, `SET CLUSTER SETTING kv.bulk_ingest.pk_buffer_size = '16MiB'`) + sqlDB.Exec(t, `SET CLUSTER SETTING kv.bulk_ingest.index_buffer_size = '16MiB'`) + srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { if r.Method == "GET" { _, _ = w.Write([]byte(r.URL.Path[1:])) From 1ef4bfee1964aebc8841f290d35331ed97fa2ab4 Mon Sep 17 00:00:00 2001 From: adityamaru Date: Thu, 8 Sep 2022 09:52:55 -0400 Subject: [PATCH 2/5] backupccl: unskip TestBackupWorkerFailure Stressed for 7+ mins without failure. 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 7e4d2cda1809..1b802f643ae9 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -8600,12 +8600,8 @@ func TestBackupOnlyPublicIndexes(t *testing.T) { func TestBackupWorkerFailure(t *testing.T) { defer leaktest.AfterTest(t)() - skip.WithIssue(t, 64773, "flaky test") defer log.Scope(t).Close(t) - skip.UnderStress(t, "under stress the test unexpectedly surfaces non-retryable errors on"+ - " backup failure") - allowResponse := make(chan struct{}) params := base.TestClusterArgs{} params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ @@ -8623,7 +8619,7 @@ func TestBackupWorkerFailure(t *testing.T) { var expectedCount int sqlDB.QueryRow(t, `SELECT count(*) FROM data.bank`).Scan(&expectedCount) - query := `BACKUP DATABASE data TO 'userfile:///worker-failure'` + query := `BACKUP DATABASE data INTO 'userfile:///worker-failure'` errCh := make(chan error) go func() { _, err := conn.Exec(query) @@ -8652,7 +8648,7 @@ func TestBackupWorkerFailure(t *testing.T) { // Drop database and restore to ensure that the backup was successful. sqlDB.Exec(t, `DROP DATABASE data`) - sqlDB.Exec(t, `RESTORE DATABASE data FROM 'userfile:///worker-failure'`) + sqlDB.Exec(t, `RESTORE DATABASE data FROM LATEST IN 'userfile:///worker-failure'`) var actualCount int sqlDB.QueryRow(t, `SELECT count(*) FROM data.bank`).Scan(&actualCount) require.Equal(t, expectedCount, actualCount) From 3de2a52dbed211b942a2a26036d06c1a12054311 Mon Sep 17 00:00:00 2001 From: adityamaru Date: Thu, 8 Sep 2022 15:40:10 -0400 Subject: [PATCH 3/5] backupccl: unskip TestBackupRestoreAppend The test was rather hard to follow because of all the different flavours of backup we were trying to test. This change removes the deprecated `BACKUP TO` statements, and also removes the `BACKUP INTO subdir IN dir` testcase that is no longer a query we allow users to run. Fixes: #54599 Release note: None --- pkg/ccl/backupccl/backup_test.go | 90 +++++++------------------------- 1 file changed, 19 insertions(+), 71 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 1b802f643ae9..7eab97129b4d 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -429,10 +429,10 @@ func TestBackupManifestFileCount(t *testing.T) { func TestBackupRestoreAppend(t *testing.T) { defer leaktest.AfterTest(t)() - skip.WithIssue(t, 54599, "flaky test") - skip.UnderRace(t, "flaky test. Issues #50984, #54599") defer log.Scope(t).Close(t) + skip.UnderStressRace(t, "test is too large to run under stress race") + const numAccounts = 1000 ctx := context.Background() tc, sqlDB, tmpDir, cleanupFn := backupRestoreTestSetup(t, multiNode, numAccounts, InitManualReplication) @@ -452,46 +452,34 @@ func TestBackupRestoreAppend(t *testing.T) { return err }) } - const localFoo1, localFoo2, localFoo3 = localFoo + "/1", localFoo + "/2", localFoo + "/3" - const userfileFoo1, userfileFoo2, userfileFoo3 = `userfile:///bar/1`, `userfile:///bar/2`, - `userfile:///bar/3` - makeBackups := func(b1, b2, b3 string) []interface{} { - return []interface{}{ - fmt.Sprintf("%s?COCKROACH_LOCALITY=%s&AUTH=implicit", b1, url.QueryEscape("default")), - fmt.Sprintf("%s?COCKROACH_LOCALITY=%s&AUTH=implicit", b2, url.QueryEscape("dc=dc1")), - fmt.Sprintf("%s?COCKROACH_LOCALITY=%s&AUTH=implicit", b3, url.QueryEscape("dc=dc2")), - } - } makeCollections := func(c1, c2, c3 string) []interface{} { return []interface{}{ - fmt.Sprintf("%s?COCKROACH_LOCALITY=%s&AUTH=implicit", c1, url.QueryEscape("default")), - fmt.Sprintf("%s?COCKROACH_LOCALITY=%s&AUTH=implicit", c2, url.QueryEscape("dc=dc1")), - fmt.Sprintf("%s?COCKROACH_LOCALITY=%s&AUTH=implicit", c3, url.QueryEscape("dc=dc2")), + fmt.Sprintf("%s?COCKROACH_LOCALITY=%s", c1, url.QueryEscape("default")), + fmt.Sprintf("%s?COCKROACH_LOCALITY=%s", c2, url.QueryEscape("dc=dc1")), + fmt.Sprintf("%s?COCKROACH_LOCALITY=%s", c3, url.QueryEscape("dc=dc2")), } } makeCollectionsWithSubdir := func(c1, c2, c3 string) []interface{} { return []interface{}{ - fmt.Sprintf("%s/%s?COCKROACH_LOCALITY=%s&AUTH=implicit", c1, "foo", url.QueryEscape("default")), - fmt.Sprintf("%s/%s?COCKROACH_LOCALITY=%s&AUTH=implicit", c2, "foo", url.QueryEscape("dc=dc1")), - fmt.Sprintf("%s/%s?COCKROACH_LOCALITY=%s&AUTH=implicit", c3, "foo", url.QueryEscape("dc=dc2")), + fmt.Sprintf("%s/%s?COCKROACH_LOCALITY=%s", c1, "foo", url.QueryEscape("default")), + fmt.Sprintf("%s/%s?COCKROACH_LOCALITY=%s", c2, "foo", url.QueryEscape("dc=dc1")), + fmt.Sprintf("%s/%s?COCKROACH_LOCALITY=%s", c3, "foo", url.QueryEscape("dc=dc2")), } } // for testing backup *into* with specified subdirectory. - const specifiedSubdir, newSpecifiedSubdir = `subdir`, `subdir2` + const specifiedSubdir = `subdir` - var full1, full2, subdirFull1, subdirFull2 string + var full1, full2 string for _, test := range []struct { name string - backups []interface{} collections []interface{} collectionsWithSubdir []interface{} }{ { "nodelocal", - makeBackups(localFoo1, localFoo2, localFoo3), // for testing backup *into* collection, pick collection shards on each // node. makeCollections(`nodelocal://0/`, `nodelocal://1/`, `nodelocal://2/`), @@ -499,56 +487,40 @@ func TestBackupRestoreAppend(t *testing.T) { }, { "userfile", - makeBackups(userfileFoo1, userfileFoo2, userfileFoo3), makeCollections(`userfile:///0`, `userfile:///1`, `userfile:///2`), makeCollectionsWithSubdir(`userfile:///0`, `userfile:///1`, `userfile:///2`), }, } { var tsBefore, ts1, ts1again, ts2 string sqlDB.QueryRow(t, "SELECT cluster_logical_timestamp()").Scan(&tsBefore) - sqlDB.Exec(t, "BACKUP TO ($1, $2, $3) AS OF SYSTEM TIME "+tsBefore, - test.backups...) sqlDB.Exec(t, "BACKUP INTO ($1, $2, $3) AS OF SYSTEM TIME "+tsBefore, test.collections...) - sqlDB.Exec(t, "BACKUP INTO $4 IN ($1, $2, $3) AS OF SYSTEM TIME "+tsBefore, - append(test.collectionsWithSubdir, specifiedSubdir)...) + + // We cannot run `BACKUP INTO subdir` when a pre-existing backup does not + // exist at that location. + sqlDB.ExpectErr(t, "A full backup cannot be written to \"/subdir\", a user defined subdirectory", + "BACKUP INTO $4 IN ($1, $2, $3) AS OF SYSTEM TIME "+tsBefore, append(test.collectionsWithSubdir, specifiedSubdir)...) sqlDB.QueryRow(t, "UPDATE data.bank SET balance = 100 RETURNING cluster_logical_timestamp()").Scan(&ts1) - sqlDB.Exec(t, "BACKUP TO ($1, $2, $3) AS OF SYSTEM TIME "+ts1, test.backups...) sqlDB.Exec(t, "BACKUP INTO LATEST IN ($1, $2, $3) AS OF SYSTEM TIME "+ts1, test.collections...) - // This should be an incremental as we already have a manifest in specifiedSubdir. - sqlDB.Exec(t, "BACKUP INTO $4 IN ($1, $2, $3) AS OF SYSTEM TIME "+ts1, - append(test.collectionsWithSubdir, specifiedSubdir)...) // Append to latest again, just to prove we can append to an appended one and // that appended didn't e.g. mess up LATEST. sqlDB.QueryRow(t, "SELECT cluster_logical_timestamp()").Scan(&ts1again) sqlDB.Exec(t, "BACKUP INTO LATEST IN ($1, $2, $3) AS OF SYSTEM TIME "+ts1again, test.collections...) - // Ensure that LATEST was created (and can be resolved) even when you backed - // up into a specified subdir to begin with. - sqlDB.Exec(t, "BACKUP INTO LATEST IN ($1, $2, $3) AS OF SYSTEM TIME "+ts1again, - test.collectionsWithSubdir...) sqlDB.QueryRow(t, "UPDATE data.bank SET balance = 200 RETURNING cluster_logical_timestamp()").Scan(&ts2) rowsTS2 := sqlDB.QueryStr(t, "SELECT * from data.bank ORDER BY id") - sqlDB.Exec(t, "BACKUP TO ($1, $2, $3) AS OF SYSTEM TIME "+ts2, test.backups...) + // Start a new full-backup in the collection version. sqlDB.Exec(t, "BACKUP INTO ($1, $2, $3) AS OF SYSTEM TIME "+ts2, test.collections...) - // Write to a new subdirectory thereby triggering a full-backup. - sqlDB.Exec(t, "BACKUP INTO $4 IN ($1, $2, $3) AS OF SYSTEM TIME "+ts2, - append(test.collectionsWithSubdir, newSpecifiedSubdir)...) sqlDB.Exec(t, "ALTER TABLE data.bank RENAME TO data.renamed") - sqlDB.Exec(t, "BACKUP TO ($1, $2, $3)", test.backups...) sqlDB.Exec(t, "BACKUP INTO LATEST IN ($1, $2, $3)", test.collections...) - sqlDB.Exec(t, "BACKUP INTO $4 IN ($1, $2, $3)", append(test.collectionsWithSubdir, - newSpecifiedSubdir)...) - sqlDB.ExpectErr(t, "cannot append a backup of specific", "BACKUP system.users TO ($1, $2, "+ - "$3)", test.backups...) // TODO(dt): prevent backing up different targets to same collection? sqlDB.Exec(t, "DROP DATABASE data CASCADE") - sqlDB.Exec(t, "RESTORE DATABASE data FROM ($1, $2, $3)", test.backups...) + sqlDB.Exec(t, "RESTORE DATABASE data FROM LATEST IN ($1, $2, $3)", test.collections...) sqlDB.ExpectErr(t, "relation \"data.bank\" does not exist", "SELECT * FROM data.bank ORDER BY id") sqlDB.CheckQueryResults(t, "SELECT * from data.renamed ORDER BY id", rowsTS2) @@ -611,37 +583,13 @@ func TestBackupRestoreAppend(t *testing.T) { })) full1 = strings.TrimSuffix(files[0], backupbase.BackupManifestName) full2 = strings.TrimSuffix(files[1], backupbase.BackupManifestName) - - // Find the full-backups written to the specified subdirectories, and within - // each also check if we can restore to individual times captured with - // incremental backups that were appended to that backup. - var subdirFiles []string - require.NoError(t, store.List(ctx, "foo/", "", func(f string) error { - ok, err := path.Match(specifiedSubdir+"*/"+backupbase.BackupManifestName, f) - if ok { - subdirFiles = append(subdirFiles, f) - } - return err - })) - require.NoError(t, err) - subdirFull1 = strings.TrimSuffix(strings.TrimPrefix(subdirFiles[0], "foo"), - backupbase.BackupManifestName) - subdirFull2 = strings.TrimSuffix(strings.TrimPrefix(subdirFiles[1], "foo"), - backupbase.BackupManifestName) } else { // Find the backup times in the collection and try RESTORE'ing to each, and // within each also check if we can restore to individual times captured with // incremental backups that were appended to that backup. full1, full2 = findFullBackupPaths(tmpDir, path.Join(tmpDir, "*/*/*/"+backupbase.BackupManifestName)) - - // Find the full-backups written to the specified subdirectories, and within - // each also check if we can restore to individual times captured with - // incremental backups that were appended to that backup. - subdirFull1, subdirFull2 = findFullBackupPaths(path.Join(tmpDir, "foo"), - path.Join(tmpDir, "foo", fmt.Sprintf("%s*", specifiedSubdir), backupbase.BackupManifestName)) } runRestores(test.collections, full1, full2) - runRestores(test.collectionsWithSubdir, subdirFull1, subdirFull2) // TODO(dt): test restoring to other backups via AOST. } @@ -1462,8 +1410,8 @@ WHERE } } - do(`BACKUP DATABASE data TO $1`, checkBackup) - do(`RESTORE data.* FROM $1 WITH OPTIONS (into_db='restoredb')`, checkRestore) + do(`BACKUP DATABASE data INTO $1`, checkBackup) + do(`RESTORE data.* FROM LATEST IN $1 WITH OPTIONS (into_db='restoredb')`, checkRestore) } func TestBackupRestoreSystemJobsProgress(t *testing.T) { From ae9224828bdba61018de698bf494c8a969cfac71 Mon Sep 17 00:00:00 2001 From: adityamaru Date: Thu, 8 Sep 2022 15:59:13 -0400 Subject: [PATCH 4/5] backupccl: delete skipped TestSpanSelectionDuringBackup This test is unweildy and not testing anything that `TestPublicIndexTableSpans` isn't. Coordinating GC and running backups is heavy handed for what we want to test and `TestPublicIndexTableSpans` offers adequate coverage that we can get rid of this. Fixes: #63209 Release note: None --- pkg/ccl/backupccl/backup_test.go | 171 ------------------------------- 1 file changed, 171 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 7eab97129b4d..d06fa9e3d03c 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -5976,177 +5976,6 @@ func TestProtectedTimestampsDuringBackup(t *testing.T) { }) } -func getTableID(db *kv.DB, dbName, tableName string) descpb.ID { - desc := desctestutils.TestingGetPublicTableDescriptor(db, keys.SystemSQLCodec, dbName, tableName) - return desc.GetID() -} - -// TestSpanSelectionDuringBackup tests the method spansForAllTableIndexes which -// is used to resolve the spans which will be backed up, and spans for which -// protected ts records will be created. -func TestProtectedTimestampSpanSelectionDuringBackup(t *testing.T) { - defer leaktest.AfterTest(t)() - skip.WithIssue(t, 63209, "flaky test") - defer log.Scope(t).Close(t) - - skip.UnderStressRace(t, - "not worth starting/stopping the server for each subtest as they all rely on the shared"+ - " variable `actualResolvedSpan`") - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - dir, dirCleanupFn := testutils.TempDir(t) - defer dirCleanupFn() - params := base.TestClusterArgs{} - params.ServerArgs.ExternalIODir = dir - var actualResolvedSpans []string - params.ServerArgs.Knobs.BackupRestore = &sql.BackupRestoreTestingKnobs{ - CaptureResolvedTableDescSpans: func(mergedSpans []roachpb.Span) { - for _, span := range mergedSpans { - actualResolvedSpans = append(actualResolvedSpans, span.String()) - } - }, - } - tc := testcluster.StartTestCluster(t, 3, params) - defer tc.Stopper().Stop(ctx) - - tc.WaitForNodeLiveness(t) - require.NoError(t, tc.WaitForFullReplication()) - - conn := tc.ServerConn(0) - runner := sqlutils.MakeSQLRunner(conn) - db := tc.Server(0).DB() - baseBackupURI := "nodelocal://0/foo/" - - t.Run("contiguous-span-merge", func(t *testing.T) { - runner.Exec(t, "CREATE DATABASE test; USE test;") - runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, "+ - "INDEX baz(name), INDEX bar (v))") - - runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) - tableID := getTableID(db, "test", "foo") - require.Equal(t, []string{fmt.Sprintf("/Table/%d/{1-4}", tableID)}, actualResolvedSpans) - runner.Exec(t, "DROP DATABASE test;") - actualResolvedSpans = nil - }) - - t.Run("drop-index-span-merge", func(t *testing.T) { - runner.Exec(t, "CREATE DATABASE test; USE test;") - runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, "+ - "INDEX baz(name), INDEX bar (v))") - runner.Exec(t, "INSERT INTO foo VALUES (1, NULL, 'testuser')") - runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=60") - runner.Exec(t, "DROP INDEX foo@baz") - - runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) - tableID := getTableID(db, "test", "foo") - require.Equal(t, []string{ - fmt.Sprintf("/Table/%d/{1-2}", tableID), - fmt.Sprintf("/Table/%d/{3-4}", tableID), - }, actualResolvedSpans) - runner.Exec(t, "DROP DATABASE test;") - actualResolvedSpans = nil - }) - - t.Run("drop-index-gced-span-merge", func(t *testing.T) { - runner.Exec(t, "CREATE DATABASE test; USE test;") - runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, "+ - "INDEX baz(name), INDEX bar (v))") - runner.Exec(t, "INSERT INTO foo VALUES (1, NULL, 'testuser')") - runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=1") - runner.Exec(t, "DROP INDEX foo@baz") - time.Sleep(time.Second * 2) - - runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) - tableID := getTableID(db, "test", "foo") - require.Equal(t, []string{ - fmt.Sprintf("/Table/%d/{1-2}", tableID), - fmt.Sprintf("/Table/%d/{3-4}", tableID), - }, actualResolvedSpans) - runner.Exec(t, "DROP DATABASE test;") - actualResolvedSpans = nil - }) - - t.Run("revs-span-merge", func(t *testing.T) { - runner.Exec(t, "CREATE DATABASE test; USE test;") - runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, "+ - "INDEX baz(name), INDEX bar (v))") - runner.Exec(t, "INSERT INTO foo VALUES (1, NULL, 'testuser')") - runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=60") - runner.Exec(t, "DROP INDEX foo@baz") - - runner.Exec(t, `BACKUP DATABASE test TO 'nodelocal://0/fooz' WITH revision_history`) - - // The BACKUP with revision history will pickup the dropped index baz as - // well because it existed in a non-drop state at some point in the interval - // covered by this BACKUP. - tableID := getTableID(db, "test", "foo") - require.Equal(t, []string{fmt.Sprintf("/Table/%d/{1-4}", tableID)}, actualResolvedSpans) - actualResolvedSpans = nil - runner.Exec(t, "DROP TABLE foo") - - runner.Exec(t, "CREATE TABLE foo2 (k INT PRIMARY KEY, v BYTES, name STRING, "+ - "INDEX baz(name), INDEX bar (v))") - runner.Exec(t, "INSERT INTO foo2 VALUES (1, NULL, 'testuser')") - runner.Exec(t, "ALTER TABLE foo2 CONFIGURE ZONE USING gc.ttlseconds=60") - runner.Exec(t, "DROP INDEX foo2@baz") - - runner.Exec(t, `BACKUP DATABASE test TO 'nodelocal://0/fooz' WITH revision_history`) - // We expect to see only the non-drop indexes of table foo in this - // incremental backup with revision history. We also expect to see both drop - // and non-drop indexes of table foo2 as all the indexes were live at some - // point in the interval covered by this BACKUP. - tableID2 := getTableID(db, "test", "foo2") - require.Equal(t, []string{ - fmt.Sprintf("/Table/%d/{1-2}", tableID), - fmt.Sprintf("/Table/%d/{3-4}", tableID), fmt.Sprintf("/Table/%d/{1-4}", tableID2), - }, - actualResolvedSpans) - runner.Exec(t, "DROP DATABASE test;") - actualResolvedSpans = nil - }) - - t.Run("last-index-dropped", func(t *testing.T) { - runner.Exec(t, "CREATE DATABASE test; USE test;") - runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, INDEX baz(name))") - runner.Exec(t, "CREATE TABLE foo2 (k INT PRIMARY KEY, v BYTES, name STRING, INDEX baz(name))") - runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=60") - runner.Exec(t, "DROP INDEX foo@baz") - - runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) - tableID := getTableID(db, "test", "foo") - tableID2 := getTableID(db, "test", "foo2") - require.Equal(t, []string{ - fmt.Sprintf("/Table/%d/{1-2}", tableID), - fmt.Sprintf("/Table/%d/{1-3}", tableID2), - }, actualResolvedSpans) - runner.Exec(t, "DROP DATABASE test;") - actualResolvedSpans = nil - }) - - t.Run("last-index-gced", func(t *testing.T) { - runner.Exec(t, "CREATE DATABASE test; USE test;") - runner.Exec(t, "CREATE TABLE foo (k INT PRIMARY KEY, v BYTES, name STRING, INDEX baz(name))") - runner.Exec(t, "INSERT INTO foo VALUES (1, NULL, 'test')") - runner.Exec(t, "CREATE TABLE foo2 (k INT PRIMARY KEY, v BYTES, name STRING, INDEX baz(name))") - runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=1") - runner.Exec(t, "DROP INDEX foo@baz") - time.Sleep(time.Second * 2) - runner.Exec(t, "ALTER TABLE foo CONFIGURE ZONE USING gc.ttlseconds=60") - - runner.Exec(t, fmt.Sprintf(`BACKUP DATABASE test INTO '%s'`, baseBackupURI+t.Name())) - tableID := getTableID(db, "test", "foo") - tableID2 := getTableID(db, "test", "foo2") - require.Equal(t, []string{ - fmt.Sprintf("/Table/%d/{1-2}", tableID), - fmt.Sprintf("/Table/%d/{1-3}", tableID2), - }, actualResolvedSpans) - runner.Exec(t, "DROP DATABASE test;") - actualResolvedSpans = nil - }) -} - func getMockIndexDesc(indexID descpb.IndexID) descpb.IndexDescriptor { mockIndexDescriptor := descpb.IndexDescriptor{ID: indexID} return mockIndexDescriptor From f79081cbae7513f99a7149917bf72a098fbc2f10 Mon Sep 17 00:00:00 2001 From: adityamaru Date: Thu, 8 Sep 2022 16:40:11 -0400 Subject: [PATCH 5/5] backupccl: unskip TestGCDropIndexSpanExpansion Ran this under stressrace, stress and race and haven't seen a single failure. Fixes: #77569 Release note: None --- pkg/ccl/backupccl/backup_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index d06fa9e3d03c..f8c6cb62792e 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -8719,8 +8719,6 @@ func TestGCDropIndexSpanExpansion(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - skip.UnderRace(t, "takes >1 min under race") - aboutToGC := make(chan struct{}) allowGC := make(chan struct{}) var gcJobID jobspb.JobID @@ -8744,6 +8742,7 @@ func TestGCDropIndexSpanExpansion(t *testing.T) { }, SkipWaitingForMVCCGC: true, }, + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), }, }}) defer tc.Stopper().Stop(ctx)