Skip to content

Commit

Permalink
Revert "backupccl: protect entire keyspan during cluster backup"
Browse files Browse the repository at this point in the history
This reverts commit 1b5fd4f.

The commit above laid a pts record over the entire table keyspace.
This did not account for two things (with the potential of there being
more):

1. System tables that we do not backup could have a short GC TTL, and
so incremental backups that attempt to protect from `StartTime` of the
previous backup would fail.

2. Dropped tables often have a short GC TTL to clear data once they have
been dropped. This change would also attempt to protect "dropped but not
gc'ed tables" even though we exclude them from the backup, and fail on
pts verification.

One suggested approach is to exclude all objects we do not backup by
subtracting these spans from {TableDataMin, TableDataMax}. This works
for system tables, and dropped but not gc'ed tables, but breaks for
dropped and gc'ed tables. A pts verification would still find the leaseholder
of the empty span and attempt to protect below the gc threshold.

In conclusion, we need to think about the semantics a little more before
we rush to protect a single keyspan.
  • Loading branch information
adityamaru committed Aug 19, 2021
1 parent a3fb75b commit 00dccf5
Show file tree
Hide file tree
Showing 3 changed files with 2 additions and 71 deletions.
1 change: 0 additions & 1 deletion pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
17 changes: 2 additions & 15 deletions pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -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
Expand Down Expand Up @@ -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
}

Expand Down
55 changes: 0 additions & 55 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
"regexp"
"strconv"
"strings"
"sync"
"sync/atomic"
"testing"
"time"
Expand All @@ -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"
Expand Down Expand Up @@ -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.
Expand Down

0 comments on commit 00dccf5

Please sign in to comment.