diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 9054ca0157bb..84acda5664e2 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -168,7 +168,6 @@ go_test( "//pkg/kv/kvclient/kvcoord:with-mocks", "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvserverbase", - "//pkg/kv/kvserver/protectedts/ptstorage", "//pkg/roachpb:with-mocks", "//pkg/scheduledjobs", "//pkg/security", diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 643ee6f5324f..c6685b1da611 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -1312,18 +1312,6 @@ func backupPlanHook( CreatedBy: backupStmt.CreatedByInfo, } - spansToProtect := spans - // If this is a full cluster backup from the system tenant then we write a - // single protected timestamp record spanning the entire keyspace. - if backupStmt.Coverage() == tree.AllDescriptors { - if p.ExecCfg().Codec.ForSystemTenant() { - spansToProtect = []roachpb.Span{{ - Key: keys.TableDataMin, - EndKey: keys.TableDataMax, - }} - } - } - if backupStmt.Options.Detached { // When running inside an explicit transaction, we simply create the job // record. We do not wait for the job to finish. @@ -1341,7 +1329,7 @@ func backupPlanHook( // The protect timestamp logic for a DETACHED BACKUP can be run within the // same txn as the BACKUP is being planned in, because we do not wait for // the BACKUP job to complete. - err = protectTimestampForBackup(ctx, p, p.ExtendedEvalContext().Txn, jobID, spansToProtect, + err = protectTimestampForBackup(ctx, p, p.ExtendedEvalContext().Txn, jobID, spans, startTime, endTime, backupDetails) if err != nil { return err @@ -1375,8 +1363,7 @@ func backupPlanHook( if err := doWriteBackupManifestCheckpoint(ctx, jobID); err != nil { return err } - if err := protectTimestampForBackup(ctx, p, plannerTxn, jobID, spansToProtect, startTime, - endTime, backupDetails); err != nil { + if err := protectTimestampForBackup(ctx, p, plannerTxn, jobID, spans, startTime, endTime, backupDetails); err != nil { return err } diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 9aa619838060..9d1585a6523d 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -26,7 +26,6 @@ import ( "regexp" "strconv" "strings" - "sync" "sync/atomic" "testing" "time" @@ -50,7 +49,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -6181,59 +6179,6 @@ func getTableID(db *kv.DB, dbName, tableName string) descpb.ID { return desc.GetID() } -func TestProtectedTimestampSpanSelectionClusterBackup(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - const numAccounts = 1 - serverArgs := base.TestServerArgs{Knobs: base.TestingKnobs{JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals()}} - params := base.TestClusterArgs{ServerArgs: serverArgs} - allowRequest := make(chan struct{}) - params.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{ - TestingRequestFilter: func(ctx context.Context, request roachpb.BatchRequest) *roachpb.Error { - for _, ru := range request.Requests { - switch ru.GetInner().(type) { - case *roachpb.ExportRequest: - <-allowRequest - } - } - return nil - }, - } - - _, tc, sqlDB, _, cleanupFn := backupRestoreTestSetupWithParams(t, singleNode, numAccounts, - InitManualReplication, params) - defer cleanupFn() - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - sqlDB.Exec(t, `BACKUP TO 'nodelocal://self/foo'`) - }() - - var jobID string - conn := tc.Conns[0] - testutils.SucceedsSoon(t, func() error { - row := conn.QueryRow("SELECT job_id FROM [SHOW JOBS] ORDER BY created DESC LIMIT 1") - return row.Scan(&jobID) - }) - - // Check protected timestamp record to ensure we are protecting the clusters - // key-span. - var spans []byte - sqlDB.QueryRow(t, `SELECT spans FROM system.protected_ts_records LIMIT 1`).Scan(&spans) - var protectedSpans ptstorage.Spans - require.NoError(t, protoutil.Unmarshal(spans, &protectedSpans)) - - expectedSpans := ptstorage.Spans{ - Spans: []roachpb.Span{{Key: keys.TableDataMin, EndKey: keys.TableDataMax}}, - } - require.Equal(t, expectedSpans, protectedSpans) - close(allowRequest) - wg.Wait() -} - // 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.