Skip to content

Commit

Permalink
backupccl: clean up old backup syntax code
Browse files Browse the repository at this point in the history
The old backup syntax was fully removed in #133610. This commit
removes all the old code that ran for the old syntax.

Epic: none

Release note: none
  • Loading branch information
kev-cao committed Nov 15, 2024
1 parent 62c52f4 commit 8bf2518
Show file tree
Hide file tree
Showing 18 changed files with 51 additions and 358 deletions.
1 change: 0 additions & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,6 @@ admission.epoch_lifo.epoch_duration duration 100ms the duration of an epoch, for
admission.epoch_lifo.queue_delay_threshold_to_switch_to_lifo duration 105ms the queue delay encountered by a (tenant,priority) for switching to epoch-LIFO ordering application
admission.sql_kv_response.enabled boolean true when true, work performed by the SQL layer when receiving a KV response is subject to admission control application
admission.sql_sql_response.enabled boolean true when true, work performed by the SQL layer when receiving a DistSQL response is subject to admission control application
bulkio.backup.deprecated_full_backup_with_subdir.enabled boolean false when true, a backup command with a user specified subdirectory will create a full backup at the subdirectory if no backup already exists at that subdirectory application
bulkio.backup.file_size byte size 128 MiB target size for individual data files produced during BACKUP application
bulkio.backup.read_timeout duration 5m0s amount of time after which a read attempt is considered timed out, which causes the backup to fail application
bulkio.backup.read_with_priority_after duration 1m0s amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads application
Expand Down
1 change: 0 additions & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@
<tr><td><div id="setting-admission-kv-enabled" class="anchored"><code>admission.kv.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>when true, work performed by the KV layer is subject to admission control</td><td>Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-admission-sql-kv-response-enabled" class="anchored"><code>admission.sql_kv_response.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>when true, work performed by the SQL layer when receiving a KV response is subject to admission control</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-admission-sql-sql-response-enabled" class="anchored"><code>admission.sql_sql_response.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>when true, work performed by the SQL layer when receiving a DistSQL response is subject to admission control</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-bulkio-backup-deprecated-full-backup-with-subdir-enabled" class="anchored"><code>bulkio.backup.deprecated_full_backup_with_subdir.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>when true, a backup command with a user specified subdirectory will create a full backup at the subdirectory if no backup already exists at that subdirectory</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-bulkio-backup-file-size" class="anchored"><code>bulkio.backup.file_size</code></div></td><td>byte size</td><td><code>128 MiB</code></td><td>target size for individual data files produced during BACKUP</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-bulkio-backup-read-timeout" class="anchored"><code>bulkio.backup.read_timeout</code></div></td><td>duration</td><td><code>5m0s</code></td><td>amount of time after which a read attempt is considered timed out, which causes the backup to fail</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
<tr><td><div id="setting-bulkio-backup-read-with-priority-after" class="anchored"><code>bulkio.backup.read_with_priority_after</code></div></td><td>duration</td><td><code>1m0s</code></td><td>amount of time since the read-as-of time above which a BACKUP should use priority when retrying reads</td><td>Serverless/Dedicated/Self-Hosted</td></tr>
Expand Down
3 changes: 1 addition & 2 deletions pkg/ccl/backupccl/alter_backup_schedule.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,8 +240,7 @@ func emitAlteredSchedule(
for i, incDest := range stmt.Options.IncrementalStorage {
incDests[i] = tree.AsStringWithFlags(incDest, tree.FmtBareStrings|tree.FmtShowFullURIs)
}
if err := emitSchedule(job, stmt, to, nil, /* incrementalFrom */
kmsURIs, incDests, resultsCh); err != nil {
if err := emitSchedule(job, stmt, to, kmsURIs, incDests, resultsCh); err != nil {
return err
}
return nil
Expand Down
11 changes: 4 additions & 7 deletions pkg/ccl/backupccl/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -614,12 +614,12 @@ func (b *backupResumer) Resume(ctx context.Context, execCtx interface{}) error {
defaultURI := details.URI
var backupDest backupdest.ResolvedDestination
if details.URI == "" {
// Choose which scheduled backup pts we will update at the the end of the
// Choose which scheduled backup pts we will update at the end of the
// backup _before_ we resolve the destination of the backup. This avoids a
// race with inc backups where backup destination resolution leads this backup
// to extend a chain that is about to be superseded by a new full backup
// chain, which could cause this inc to accidentally push the pts for the
// _new_ chain instead of the old chain it is apart of. By choosing the pts to
// _new_ chain instead of the old chain it is a part of. By choosing the pts to
// move before we resolve the destination, we guarantee that we push the old
// chain.
insqlDB := p.ExecCfg().InternalDB
Expand All @@ -630,8 +630,7 @@ func (b *backupResumer) Resume(ctx context.Context, execCtx interface{}) error {
}

var err error
backupDest, err = backupdest.ResolveDest(ctx, p.User(), details.Destination, details.EndTime,
details.IncrementalFrom, p.ExecCfg())
backupDest, err = backupdest.ResolveDest(ctx, p.User(), details.Destination, details.EndTime, p.ExecCfg())
if err != nil {
return err
}
Expand Down Expand Up @@ -1069,9 +1068,7 @@ func collectTelemetry(
countSource("backup.span.incremental")
telemetry.CountBucketed("backup.incremental-span-sec",
int64(backupDetails.EndTime.GoTime().Sub(backupDetails.StartTime.GoTime()).Seconds()))
if len(initialDetails.IncrementalFrom) == 0 {
countSource("backup.auto-incremental")
}
countSource("backup.auto-incremental")
}
if len(backupDetails.URIsByLocalityKV) > 1 {
countSource("backup.partitioned")
Expand Down
54 changes: 12 additions & 42 deletions pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,6 @@ func resolveOptionsForBackupJobDescription(
func GetRedactedBackupNode(
backup *tree.Backup,
to []string,
incrementalFrom []string,
kmsURIs []string,
resolvedSubdir string,
incrementalStorage []string,
Expand All @@ -113,7 +112,6 @@ func GetRedactedBackupNode(
b := &tree.Backup{
AsOf: backup.AsOf,
Targets: backup.Targets,
Nested: backup.Nested,
AppendToLatest: backup.AppendToLatest,
}

Expand All @@ -125,7 +123,7 @@ func GetRedactedBackupNode(
// LATEST, where we are appending an incremental BACKUP.
// - For `BACKUP INTO x` this would be the sub-directory we have selected to
// write the BACKUP to.
if b.Nested && hasBeenPlanned {
if hasBeenPlanned {
b.Subdir = tree.NewDString(resolvedSubdir)
}

Expand All @@ -135,11 +133,6 @@ func GetRedactedBackupNode(
return nil, err
}

b.IncrementalFrom, err = sanitizeURIList(incrementalFrom)
if err != nil {
return nil, err
}

resolvedOpts, err := resolveOptionsForBackupJobDescription(backup.Options, kmsURIs,
incrementalStorage)
if err != nil {
Expand All @@ -166,12 +159,11 @@ func backupJobDescription(
p sql.PlanHookState,
backup *tree.Backup,
to []string,
incrementalFrom []string,
kmsURIs []string,
resolvedSubdir string,
incrementalStorage []string,
) (string, error) {
b, err := GetRedactedBackupNode(backup, to, incrementalFrom, kmsURIs,
b, err := GetRedactedBackupNode(backup, to, kmsURIs,
resolvedSubdir, incrementalStorage, true /* hasBeenPlanned */)
if err != nil {
return "", err
Expand Down Expand Up @@ -406,7 +398,6 @@ func backupTypeCheck(
},
exprutil.StringArrays{
tree.Exprs(backupStmt.To),
backupStmt.IncrementalFrom,
tree.Exprs(backupStmt.Options.IncrementalStorage),
tree.Exprs(backupStmt.Options.EncryptionKMSURI),
},
Expand Down Expand Up @@ -439,16 +430,6 @@ func backupPlanHook(

detached := backupStmt.Options.Detached == tree.DBoolTrue

// Deprecation notice for `BACKUP TO` syntax. Remove this once the syntax is
// deleted in 22.2.
if !backupStmt.Nested {
p.BufferClientNotice(ctx,
pgnotice.Newf("The `BACKUP TO` syntax will be removed in a future release, please"+
" switch over to using `BACKUP INTO` to create a backup collection: %s. "+
"Backups created using the `BACKUP TO` syntax may not be restoreable in the next major version release.",
"https://www.cockroachlabs.com/docs/stable/backup.html#considerations"))
}

exprEval := p.ExprEvaluator("BACKUP")

var err error
Expand All @@ -464,10 +445,6 @@ func backupPlanHook(
if err != nil {
return nil, nil, nil, false, err
}
incrementalFrom, err := exprEval.StringArray(ctx, backupStmt.IncrementalFrom)
if err != nil {
return nil, nil, nil, false, err
}

incrementalStorage, err := exprEval.StringArray(
ctx, tree.Exprs(backupStmt.Options.IncrementalStorage),
Expand Down Expand Up @@ -565,9 +542,6 @@ func backupPlanHook(
}
}

if !backupStmt.Nested && len(incrementalStorage) > 0 {
return errors.New("incremental_location option not supported with `BACKUP TO` syntax")
}
if len(incrementalStorage) > 0 && (len(incrementalStorage) != len(to)) {
return errors.New("the incremental_location option must contain the same number of locality" +
" aware URIs as the full backup destination")
Expand Down Expand Up @@ -662,7 +636,6 @@ func backupPlanHook(
EndTime: endTime,
RevisionHistory: revisionHistory,
IncludeAllSecondaryTenants: includeAllSecondaryTenants,
IncrementalFrom: incrementalFrom,
FullCluster: backupStmt.Coverage() == tree.AllDescriptors,
ResolvedCompleteDbs: completeDBs,
EncryptionOptions: &encryptionParams,
Expand Down Expand Up @@ -695,17 +668,14 @@ func backupPlanHook(
}
}

if backupStmt.Nested {
if backupStmt.AppendToLatest {
initialDetails.Destination.Subdir = backupbase.LatestFileName
initialDetails.Destination.Exists = true

} else if subdir != "" {
initialDetails.Destination.Subdir = "/" + strings.TrimPrefix(subdir, "/")
initialDetails.Destination.Exists = true
} else {
initialDetails.Destination.Subdir = endTime.GoTime().Format(backupbase.DateBasedIntoFolderName)
}
if backupStmt.AppendToLatest {
initialDetails.Destination.Subdir = backupbase.LatestFileName
initialDetails.Destination.Exists = true
} else if subdir != "" {
initialDetails.Destination.Subdir = "/" + strings.TrimPrefix(subdir, "/")
initialDetails.Destination.Exists = true
} else {
initialDetails.Destination.Subdir = endTime.GoTime().Format(backupbase.DateBasedIntoFolderName)
}

if backupStmt.Targets != nil && backupStmt.Targets.TenantID.IsSet() {
Expand All @@ -721,12 +691,12 @@ func backupPlanHook(

jobID := p.ExecCfg().JobRegistry.MakeJobID()

if err := logAndSanitizeBackupDestinations(ctx, append(to, incrementalFrom...)...); err != nil {
if err := logAndSanitizeBackupDestinations(ctx, to...); err != nil {
return errors.Wrap(err, "logging backup destinations")
}

description, err := backupJobDescription(p,
backupStmt.Backup, to, incrementalFrom,
backupStmt.Backup, to,
encryptionParams.RawKmsUris,
initialDetails.Destination.Subdir,
initialDetails.Destination.IncrementalStorage,
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -723,7 +723,7 @@ func TestBackupAndRestoreJobDescription(t *testing.T) {
"BACKUP INTO LATEST IN $4 WITH incremental_location=($1, $2, $3)",
append(incrementals, collections[0])...)

sqlDB.ExpectErr(t, "A full backup cannot be written to \"/subdir\", a user defined subdirectory. To take a full backup, remove the subdirectory from the backup command",
sqlDB.ExpectErr(t, "No full backup exists in \"/subdir\" to append an incremental backup to. To take a full backup, remove the subdirectory from the backup command",
"BACKUP INTO $4 IN ($1, $2, $3)", append(collections, "subdir")...)

time.Sleep(time.Second + 2)
Expand Down
2 changes: 0 additions & 2 deletions pkg/ccl/backupccl/backupdest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ go_library(
"//pkg/jobs/jobspb",
"//pkg/roachpb",
"//pkg/security/username",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/pgwire/pgcode",
Expand Down Expand Up @@ -64,7 +63,6 @@ go_test(
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/randutil",
"//pkg/util/timeutil",
"@com_github_stretchr_testify//require",
],
)
70 changes: 12 additions & 58 deletions pkg/ccl/backupccl/backupdest/backup_destination.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
Expand Down Expand Up @@ -52,17 +51,6 @@ const (
// will contain one.
var backupPathRE = regexp.MustCompile("^/?[^\\/]+/[^\\/]+/[^\\/]+/" + backupbase.BackupManifestName + "$")

// featureFullBackupUserSubdir, when true, will create a full backup at a user
// specified subdirectory if no backup already exists at that subdirectory. As
// of 22.1, this feature is default disabled, and will be totally disabled by 22.2.
var featureFullBackupUserSubdir = settings.RegisterBoolSetting(
settings.ApplicationLevel,
"bulkio.backup.deprecated_full_backup_with_subdir.enabled",
"when true, a backup command with a user specified subdirectory will create a full backup at"+
" the subdirectory if no backup already exists at that subdirectory",
false,
settings.WithPublic)

// TODO(adityamaru): Move this to the soon to be `backupinfo` package.
func containsManifest(ctx context.Context, exportStore cloud.ExternalStorage) (bool, error) {
r, _, err := exportStore.ReadFile(ctx, backupbase.BackupManifestName, cloud.ReadOptions{NoFileSize: true})
Expand Down Expand Up @@ -113,7 +101,6 @@ func ResolveDest(
user username.SQLUsername,
dest jobspb.BackupDetails_Destination,
endTime hlc.Timestamp,
incrementalFrom []string,
execCfg *sql.ExecutorConfig,
) (ResolvedDestination, error) {
makeCloudStorage := execCfg.DistSQLSrv.ExternalStorageFromURI
Expand All @@ -125,39 +112,21 @@ func ResolveDest(

var collectionURI string
chosenSuffix := dest.Subdir
if chosenSuffix != "" {
// The legacy backup syntax, BACKUP TO, leaves the dest.Subdir and collection parameters empty.
collectionURI = defaultURI
collectionURI = defaultURI

if chosenSuffix == backupbase.LatestFileName {
latest, err := ReadLatestFile(ctx, defaultURI, makeCloudStorage, user)
if err != nil {
return ResolvedDestination{}, err
}
chosenSuffix = latest
if chosenSuffix == backupbase.LatestFileName {
latest, err := ReadLatestFile(ctx, defaultURI, makeCloudStorage, user)
if err != nil {
return ResolvedDestination{}, err
}
chosenSuffix = latest
}

plannedBackupDefaultURI, urisByLocalityKV, err := GetURIsByLocalityKV(dest.To, chosenSuffix)
if err != nil {
return ResolvedDestination{}, err
}

// At this point, the plannedBackupDefaultURI is the full path for the backup. For BACKUP
// INTO, this path includes the chosenSuffix. Once this function returns, the
// plannedBackupDefaultURI will be the full path for this backup in planning.
if len(incrementalFrom) != 0 {
// Legacy backup with deprecated BACKUP TO-syntax.
prevBackupURIs := incrementalFrom
return ResolvedDestination{
CollectionURI: collectionURI,
DefaultURI: plannedBackupDefaultURI,
ChosenSubdir: chosenSuffix,
URIsByLocalityKV: urisByLocalityKV,
PrevBackupURIs: prevBackupURIs,
}, nil
}

defaultStore, err := makeCloudStorage(ctx, plannedBackupDefaultURI, user)
if err != nil {
return ResolvedDestination{}, err
Expand All @@ -167,35 +136,20 @@ func ResolveDest(
if err != nil {
return ResolvedDestination{}, err
}
if exists && !dest.Exists && chosenSuffix != "" {
if exists && !dest.Exists {
// We disallow a user from writing a full backup to a path in a collection containing an
// existing backup iff we're 99.9% confident this backup was planned on a 22.1 node.
return ResolvedDestination{},
errors.Newf("A full backup already exists in %s. "+
"Consider running an incremental backup to this full backup via `BACKUP INTO '%s' IN '%s'`",
"Consider running an incremental backup on this full backup via `BACKUP INTO '%s' IN '%s'`",
plannedBackupDefaultURI, chosenSuffix, dest.To[0])

} else if !exists {
if dest.Exists {
// Implies the user passed a subdirectory in their backup command, either
// explicitly or using LATEST; however, we could not find an existing
// backup in that subdirectory.
// - Pre 22.1: this was fine. we created a full backup in their specified subdirectory.
// - 22.1: throw an error: full backups with an explicit subdirectory are deprecated.
// User can use old behavior by switching the 'bulkio.backup.full_backup_with_subdir.
// enabled' to true.
// - 22.2+: the backup will fail unconditionally.
// TODO (msbutler): throw error in 22.2
if !featureFullBackupUserSubdir.Get(execCfg.SV()) {
return ResolvedDestination{},
errors.Errorf("A full backup cannot be written to %q, a user defined subdirectory. "+
"To take a full backup, remove the subdirectory from the backup command "+
"(i.e. run 'BACKUP ... INTO <collectionURI>'). "+
"Or, to take a full backup at a specific subdirectory, "+
"enable the deprecated syntax by switching the %q cluster setting to true; "+
"however, note this deprecated syntax will not be available in a future release.",
chosenSuffix, featureFullBackupUserSubdir.Name())
}
return ResolvedDestination{},
errors.Errorf("No full backup exists in %q to append an incremental backup to. "+
"To take a full backup, remove the subdirectory from the backup command "+
"(i.e. run 'BACKUP ... INTO <collectionURI>'). ", chosenSuffix)
}
// There's no full backup in the resolved subdirectory; therefore, we're conducting a full backup.
return ResolvedDestination{
Expand Down
Loading

0 comments on commit 8bf2518

Please sign in to comment.