From 8bf251845d152f6ab01792eb2aaa9c35f518a820 Mon Sep 17 00:00:00 2001 From: Kevin Cao Date: Tue, 12 Nov 2024 13:36:42 -0500 Subject: [PATCH] backupccl: clean up old backup syntax code 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 --- .../settings/settings-for-tenants.txt | 1 - docs/generated/settings/settings.html | 1 - pkg/ccl/backupccl/alter_backup_schedule.go | 3 +- pkg/ccl/backupccl/backup_job.go | 11 +- pkg/ccl/backupccl/backup_planning.go | 54 ++---- pkg/ccl/backupccl/backup_test.go | 2 +- pkg/ccl/backupccl/backupdest/BUILD.bazel | 2 - .../backupdest/backup_destination.go | 70 ++------ .../backupdest/backup_destination_test.go | 165 ------------------ pkg/ccl/backupccl/backuppb/backup.go | 14 -- pkg/ccl/backupccl/create_scheduled_backup.go | 11 +- pkg/ccl/backupccl/schedule_exec.go | 1 - pkg/internal/sqlsmith/bulkio.go | 1 - pkg/jobs/jobspb/jobs.proto | 2 +- pkg/sql/parser/sql.y | 3 - pkg/sql/sem/tree/backup.go | 38 +--- pkg/sql/sem/tree/pretty.go | 19 +- pkg/sql/sem/tree/walk.go | 11 +- 18 files changed, 51 insertions(+), 358 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 4de8d6edbd56..1ade2de05c73 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -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 diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 40476e3cd6c4..01a0aa56d3a1 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -9,7 +9,6 @@
admission.kv.enabled
booleantruewhen true, work performed by the KV layer is subject to admission controlDedicated/Self-Hosted
admission.sql_kv_response.enabled
booleantruewhen true, work performed by the SQL layer when receiving a KV response is subject to admission controlServerless/Dedicated/Self-Hosted
admission.sql_sql_response.enabled
booleantruewhen true, work performed by the SQL layer when receiving a DistSQL response is subject to admission controlServerless/Dedicated/Self-Hosted -
bulkio.backup.deprecated_full_backup_with_subdir.enabled
booleanfalsewhen true, a backup command with a user specified subdirectory will create a full backup at the subdirectory if no backup already exists at that subdirectoryServerless/Dedicated/Self-Hosted
bulkio.backup.file_size
byte size128 MiBtarget size for individual data files produced during BACKUPServerless/Dedicated/Self-Hosted
bulkio.backup.read_timeout
duration5m0samount of time after which a read attempt is considered timed out, which causes the backup to failServerless/Dedicated/Self-Hosted
bulkio.backup.read_with_priority_after
duration1m0samount of time since the read-as-of time above which a BACKUP should use priority when retrying readsServerless/Dedicated/Self-Hosted diff --git a/pkg/ccl/backupccl/alter_backup_schedule.go b/pkg/ccl/backupccl/alter_backup_schedule.go index 481244ca670b..838033466997 100644 --- a/pkg/ccl/backupccl/alter_backup_schedule.go +++ b/pkg/ccl/backupccl/alter_backup_schedule.go @@ -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 diff --git a/pkg/ccl/backupccl/backup_job.go b/pkg/ccl/backupccl/backup_job.go index 9cb29813904d..0329b93cc42d 100644 --- a/pkg/ccl/backupccl/backup_job.go +++ b/pkg/ccl/backupccl/backup_job.go @@ -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 @@ -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 } @@ -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") diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index bdd904d01c95..08e77ad8e758 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -104,7 +104,6 @@ func resolveOptionsForBackupJobDescription( func GetRedactedBackupNode( backup *tree.Backup, to []string, - incrementalFrom []string, kmsURIs []string, resolvedSubdir string, incrementalStorage []string, @@ -113,7 +112,6 @@ func GetRedactedBackupNode( b := &tree.Backup{ AsOf: backup.AsOf, Targets: backup.Targets, - Nested: backup.Nested, AppendToLatest: backup.AppendToLatest, } @@ -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) } @@ -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 { @@ -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 @@ -406,7 +398,6 @@ func backupTypeCheck( }, exprutil.StringArrays{ tree.Exprs(backupStmt.To), - backupStmt.IncrementalFrom, tree.Exprs(backupStmt.Options.IncrementalStorage), tree.Exprs(backupStmt.Options.EncryptionKMSURI), }, @@ -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 @@ -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), @@ -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") @@ -662,7 +636,6 @@ func backupPlanHook( EndTime: endTime, RevisionHistory: revisionHistory, IncludeAllSecondaryTenants: includeAllSecondaryTenants, - IncrementalFrom: incrementalFrom, FullCluster: backupStmt.Coverage() == tree.AllDescriptors, ResolvedCompleteDbs: completeDBs, EncryptionOptions: &encryptionParams, @@ -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() { @@ -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, diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 76d53c7b35d2..86114872c058 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -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) diff --git a/pkg/ccl/backupccl/backupdest/BUILD.bazel b/pkg/ccl/backupccl/backupdest/BUILD.bazel index b93b66b78182..e14ff5395f2c 100644 --- a/pkg/ccl/backupccl/backupdest/BUILD.bazel +++ b/pkg/ccl/backupccl/backupdest/BUILD.bazel @@ -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", @@ -64,7 +63,6 @@ go_test( "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/randutil", - "//pkg/util/timeutil", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/ccl/backupccl/backupdest/backup_destination.go b/pkg/ccl/backupccl/backupdest/backup_destination.go index 21200a935fc8..b501a587bbda 100644 --- a/pkg/ccl/backupccl/backupdest/backup_destination.go +++ b/pkg/ccl/backupccl/backupdest/backup_destination.go @@ -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" @@ -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}) @@ -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 @@ -125,17 +112,14 @@ 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) @@ -143,21 +127,6 @@ func ResolveDest( 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 @@ -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 '). "+ - "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 '). ", chosenSuffix) } // There's no full backup in the resolved subdirectory; therefore, we're conducting a full backup. return ResolvedDestination{ diff --git a/pkg/ccl/backupccl/backupdest/backup_destination_test.go b/pkg/ccl/backupccl/backupdest/backup_destination_test.go index dad5ce5347f9..5eef0eb74223 100644 --- a/pkg/ccl/backupccl/backupdest/backup_destination_test.go +++ b/pkg/ccl/backupccl/backupdest/backup_destination_test.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/stretchr/testify/require" ) @@ -93,168 +92,6 @@ func TestBackupRestoreResolveDestination(t *testing.T) { localities = []string{"dc=EN", "dc=FR"} } t.Run(fmt.Sprintf("locality-aware-%t", localityAware), func(t *testing.T) { - - // When testing explicit backup locations, we'll be testing the name - // resolution on backup directory structures created when running a - // sequence of backups like: - // - BACKUP TO full - // - BACKUP TO inc1 INCREMENTAL FROM full - // - BACKUP TO inc1 INCREMENTAL FROM full, inc1 - // - // We write backup manifests as we test as if we were actually running the - // backup. - t.Run("explicit", func(t *testing.T) { - fullLoc := fmt.Sprintf("nodelocal://1/%s?AUTH=implicit", t.Name()) - inc1Loc := fmt.Sprintf("nodelocal://1/%s/inc1?AUTH=implicit", t.Name()) - inc2Loc := fmt.Sprintf("nodelocal://1/%s/inc2?AUTH=implicit", t.Name()) - - testExplicitBackup := func(t *testing.T, to []string, incrementalFrom []string) { - // Time doesn't matter for these since we don't create any date-based - // subdirectory. Let's just use now. - endTime := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} - - expectedPrevBackups := make([]string, len(incrementalFrom)) - for i, incrementalLoc := range incrementalFrom { - expectedPrevBackups[i] = localizeURI(t, incrementalLoc, localities)[0] - } - defaultDest, localitiesDest, err := backupdest.GetURIsByLocalityKV(to, "") - require.NoError(t, err) - - backupDest, err := backupdest.ResolveDest( - ctx, username.RootUserName(), - jobspb.BackupDetails_Destination{To: to}, - endTime, - incrementalFrom, - &execCfg, - ) - require.NoError(t, err) - - // Not an INTO backup, so no collection of suffix info. - require.Equal(t, "", backupDest.CollectionURI) - require.Equal(t, "", backupDest.ChosenSubdir) - - require.Equal(t, defaultDest, backupDest.DefaultURI) - require.Equal(t, localitiesDest, backupDest.URIsByLocalityKV) - require.Equal(t, incrementalFrom, backupDest.PrevBackupURIs) - } - - // The first initial full backup: BACKUP TO full. - { - incrementalFrom := []string(nil) - to := localizeURI(t, fullLoc, localities) - testExplicitBackup(t, to, incrementalFrom) - - // Write the manifest files as if this backup succeeded. - writeManifest(t, fullLoc) - } - - // An incremental on top if it: BACKUP TO inc1 INCREMENTAL FROM full. - { - incrementalFrom := []string{fullLoc} - to := localizeURI(t, inc1Loc, localities) - testExplicitBackup(t, to, incrementalFrom) - - // Write the manifest files as if this backup succeeded. - writeManifest(t, inc1Loc) - } - - // Another incremental on top of the incremental: BACKUP TO inc2 - // INCREMENTAL FROM full, inc1. - { - incrementalFrom := []string{fullLoc, inc1Loc} - to := localizeURI(t, inc2Loc, localities) - testExplicitBackup(t, to, incrementalFrom) - - writeManifest(t, inc2Loc) - } - }) - - // When testing auto-append backup locations, we'll be testing the name - // resolution on backup directory structures created when running a sequence - // of backups like: - // - BACKUP TO full - // - BACKUP TO full - // - BACKUP TO full - t.Run("auto-append", func(t *testing.T) { - baseDir := fmt.Sprintf("nodelocal://1/%s?AUTH=implicit", t.Name()) - fullTime := time.Date(2020, 12, 25, 6, 0, 0, 0, time.UTC) - inc1Time := fullTime.Add(time.Minute * 30) - inc2Time := inc1Time.Add(time.Minute * 30) - prevBackups := []string(nil) - - testAutoAppendBackup := func(t *testing.T, to []string, backupTime time.Time, - expectedDefault string, expectedLocalities map[string]string, expectedPrevBackups []string, - ) { - endTime := hlc.Timestamp{WallTime: backupTime.UnixNano()} - - backupDest, err := backupdest.ResolveDest( - ctx, username.RootUserName(), - jobspb.BackupDetails_Destination{To: to}, - endTime, - nil, /* incrementalFrom */ - &execCfg, - ) - require.NoError(t, err) - - // Not a backup collection. - require.Equal(t, "", backupDest.CollectionURI) - require.Equal(t, "", backupDest.ChosenSubdir) - require.Equal(t, expectedDefault, backupDest.DefaultURI) - require.Equal(t, expectedLocalities, backupDest.URIsByLocalityKV) - require.Equal(t, expectedPrevBackups, backupDest.PrevBackupURIs) - } - - // Initial full backup: BACKUP TO baseDir. - { - to := localizeURI(t, baseDir, localities) - // The full backup should go into the baseDir. - expectedDefault := baseDir - expectedLocalities := make(map[string]string) - for _, locality := range localities { - expectedLocalities[locality] = fmt.Sprintf("nodelocal://1/%s/%s?AUTH=implicit", t.Name(), locality) - } - - testAutoAppendBackup(t, to, fullTime, expectedDefault, expectedLocalities, prevBackups) - - prevBackups = append(prevBackups, expectedDefault) - writeManifest(t, expectedDefault) - } - - // Incremental: BACKUP TO baseDir. - { - to := localizeURI(t, baseDir, localities) - // The full backup should go into the baseDir. - expectedDefault := fmt.Sprintf("nodelocal://1/%s/incrementals/20201225/063000.00?AUTH=implicit", t.Name()) - expectedLocalities := make(map[string]string) - for _, locality := range localities { - expectedLocalities[locality] = fmt.Sprintf("nodelocal://1/%s/%s/incrementals/20201225/063000.00?AUTH=implicit", t.Name(), locality) - } - - testAutoAppendBackup(t, to, inc1Time, expectedDefault, expectedLocalities, prevBackups) - - prevBackups = append(prevBackups, expectedDefault) - writeManifest(t, expectedDefault) - } - - // Another incremental: BACKUP TO baseDir. - { - to := localizeURI(t, baseDir, localities) - // We expect another incremental to go into the appropriate time - // formatted sub-directory. - expectedDefault := fmt.Sprintf( - "nodelocal://1/%s/incrementals/20201225/070000.00?AUTH=implicit", t.Name()) - expectedLocalities := make(map[string]string) - for _, locality := range localities { - expectedLocalities[locality] = fmt.Sprintf( - "nodelocal://1/%s/%s/incrementals/20201225/070000.00?AUTH=implicit", - t.Name(), locality) - } - - testAutoAppendBackup(t, to, inc2Time, expectedDefault, expectedLocalities, prevBackups) - writeManifest(t, expectedDefault) - } - }) - // When testing auto-append backup locations, we'll be testing the name // resolution on backup directory structures created when running a sequence // of backups like: @@ -313,7 +150,6 @@ func TestBackupRestoreResolveDestination(t *testing.T) { appendToLatest bool, subdir string, incrementalTo []string) { endTime := hlc.Timestamp{WallTime: backupTime.UnixNano()} - incrementalFrom := []string(nil) if appendToLatest { subdir = backupbase.LatestFileName @@ -338,7 +174,6 @@ func TestBackupRestoreResolveDestination(t *testing.T) { jobspb.BackupDetails_Destination{To: collectionTo, Subdir: subdir, IncrementalStorage: incrementalTo, Exists: fullBackupExists}, endTime, - incrementalFrom, &execCfg, ) require.NoError(t, err) diff --git a/pkg/ccl/backupccl/backuppb/backup.go b/pkg/ccl/backupccl/backuppb/backup.go index 5e77909accef..ee40af124b81 100644 --- a/pkg/ccl/backupccl/backuppb/backup.go +++ b/pkg/ccl/backupccl/backuppb/backup.go @@ -117,20 +117,6 @@ func (m ScheduledBackupExecutionArgs) MarshalJSONPB(marshaller *jsonpb.Marshaler backup.To[i] = tree.NewDString(clean) } - // NB: this will never be non-nil with current schedule syntax but is here for - // completeness. - for i := range backup.IncrementalFrom { - raw, ok := backup.IncrementalFrom[i].(*tree.StrVal) - if !ok { - return nil, errors.Errorf("unexpected %T arg in backup schedule: %v", raw, raw) - } - clean, err := cloud.SanitizeExternalStorageURI(raw.RawString(), nil /* extraParams */) - if err != nil { - return nil, err - } - backup.IncrementalFrom[i] = tree.NewDString(clean) - } - for i := range backup.Options.IncrementalStorage { raw, ok := backup.Options.IncrementalStorage[i].(*tree.StrVal) if !ok { diff --git a/pkg/ccl/backupccl/create_scheduled_backup.go b/pkg/ccl/backupccl/create_scheduled_backup.go index 40a1c4ea7128..e5090efdc537 100644 --- a/pkg/ccl/backupccl/create_scheduled_backup.go +++ b/pkg/ccl/backupccl/create_scheduled_backup.go @@ -221,7 +221,6 @@ func doCreateBackupSchedules( Options: tree.BackupOptions{ Detached: tree.DBoolTrue, }, - Nested: true, AppendToLatest: false, } @@ -361,8 +360,7 @@ func doCreateBackupSchedules( if err := scheduledJobs.Create(ctx, inc); err != nil { return err } - if err := emitSchedule(inc, backupNode, destinations, nil, /* incrementalFrom */ - kmsURIs, incDests, resultsCh); err != nil { + if err := emitSchedule(inc, backupNode, destinations, kmsURIs, incDests, resultsCh); err != nil { return err } unpauseOnSuccessID = inc.ScheduleID() @@ -413,8 +411,7 @@ func doCreateBackupSchedules( } collectScheduledBackupTelemetry(ctx, incRecurrence, fullRecurrence, firstRun, fullRecurrencePicked, ignoreExisting, details, backupEvent) - return emitSchedule(full, backupNode, destinations, nil, /* incrementalFrom */ - kmsURIs, nil, resultsCh) + return emitSchedule(full, backupNode, destinations, kmsURIs, nil, resultsCh) } func setDependentSchedule( @@ -521,7 +518,7 @@ func makeBackupSchedule( func emitSchedule( sj *jobs.ScheduledJob, backupNode *tree.Backup, - to, incrementalFrom, kmsURIs []string, + to, kmsURIs []string, incrementalStorage []string, resultsCh chan<- tree.Datums, ) error { @@ -541,7 +538,7 @@ func emitSchedule( nextRun = next } - redactedBackupNode, err := GetRedactedBackupNode(backupNode, to, incrementalFrom, kmsURIs, "", + redactedBackupNode, err := GetRedactedBackupNode(backupNode, to, kmsURIs, "", incrementalStorage, false /* hasBeenPlanned */) if err != nil { return err diff --git a/pkg/ccl/backupccl/schedule_exec.go b/pkg/ccl/backupccl/schedule_exec.go index 7dd54f824cb3..9d34171a3f2e 100644 --- a/pkg/ccl/backupccl/schedule_exec.go +++ b/pkg/ccl/backupccl/schedule_exec.go @@ -321,7 +321,6 @@ func (e *scheduledBackupExecutor) GetCreateScheduleStatement( redactedBackupNode, err := GetRedactedBackupNode( backupNode.Backup, destinations, - nil, /* incrementalFrom */ kmsURIs, "", nil, diff --git a/pkg/internal/sqlsmith/bulkio.go b/pkg/internal/sqlsmith/bulkio.go index ee02ae9751b7..46b02816ecd0 100644 --- a/pkg/internal/sqlsmith/bulkio.go +++ b/pkg/internal/sqlsmith/bulkio.go @@ -111,7 +111,6 @@ func makeBackup(s *Smither) (tree.Statement, bool) { } return &tree.Backup{ - Nested: true, Targets: &targets, To: tree.StringOrPlaceholderOptList{tree.NewStrVal(name)}, AsOf: makeAsOf(s), diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index bebfe9eeaa68..0725ccae13d9 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -398,7 +398,7 @@ message BackupDetails { SchedulePTSChainingRecord schedule_pts_chaining_record = 10 [(gogoproto.customname) = "SchedulePTSChainingRecord"]; bool revision_history = 13; - repeated string incremental_from = 14; + reserved 14; bool full_cluster = 15; reserved 16; diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 88dbc021da01..e47099bbaed2 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -3359,7 +3359,6 @@ backup_stmt: $$.val = &tree.Backup{ Targets: $2.backupTargetListPtr(), To: $6.stringOrPlaceholderOptList(), - Nested: true, AppendToLatest: false, Subdir: $4.expr(), AsOf: $7.asOfClause(), @@ -3371,7 +3370,6 @@ backup_stmt: $$.val = &tree.Backup{ Targets: $2.backupTargetListPtr(), To: $4.stringOrPlaceholderOptList(), - Nested: true, AsOf: $5.asOfClause(), Options: *$6.backupOptions(), } @@ -3381,7 +3379,6 @@ backup_stmt: $$.val = &tree.Backup{ Targets: $2.backupTargetListPtr(), To: $6.stringOrPlaceholderOptList(), - Nested: true, AppendToLatest: true, AsOf: $7.asOfClause(), Options: *$8.backupOptions(), diff --git a/pkg/sql/sem/tree/backup.go b/pkg/sql/sem/tree/backup.go index ac285eb38bd5..6f7af7c21391 100644 --- a/pkg/sql/sem/tree/backup.go +++ b/pkg/sql/sem/tree/backup.go @@ -55,24 +55,17 @@ type Backup struct { // the docs). To StringOrPlaceholderOptList - // IncrementalFrom is only set for the old 'BACKUP .... TO ...' syntax. - IncrementalFrom Exprs - AsOf AsOfClause Options BackupOptions - // Nested is set to true when the user creates a backup with - //`BACKUP ... INTO... ` syntax. - Nested bool - // AppendToLatest is set to true if the user creates a backup with //`BACKUP...INTO LATEST...` AppendToLatest bool // Subdir may be set by the parser when the SQL query is of the form `BACKUP - // INTO 'subdir' IN...`. Alternatively, if Nested is true but a subdir was not - // explicitly specified by the user, then this will be set during BACKUP - // planning once the destination has been resolved. + // INTO 'subdir' IN...`. Alternatively, if a subdir was not explicitly specified + // by the user, then this will be set during BACKUP planning once the destination + // has been resolved. Subdir Expr } @@ -85,31 +78,18 @@ func (node *Backup) Format(ctx *FmtCtx) { ctx.FormatNode(node.Targets) ctx.WriteString(" ") } - if node.Nested { - ctx.WriteString("INTO ") - if node.Subdir != nil { - ctx.FormatNode(node.Subdir) - ctx.WriteString(" IN ") - } else if node.AppendToLatest { - ctx.WriteString("LATEST IN ") - } - } else { - ctx.WriteString("TO ") + ctx.WriteString("INTO ") + if node.Subdir != nil { + ctx.FormatNode(node.Subdir) + ctx.WriteString(" IN ") + } else if node.AppendToLatest { + ctx.WriteString("LATEST IN ") } ctx.FormatURIs(node.To) if node.AsOf.Expr != nil { ctx.WriteString(" ") ctx.FormatNode(&node.AsOf) } - if node.IncrementalFrom != nil { - ctx.WriteString(" INCREMENTAL FROM ") - for i, from := range node.IncrementalFrom { - if i > 0 { - ctx.WriteString(", ") - } - ctx.FormatURI(from) - } - } if !node.Options.IsDefault() { ctx.WriteString(" WITH OPTIONS (") diff --git a/pkg/sql/sem/tree/pretty.go b/pkg/sql/sem/tree/pretty.go index 08cec60536ca..ab359f1d671c 100644 --- a/pkg/sql/sem/tree/pretty.go +++ b/pkg/sql/sem/tree/pretty.go @@ -2155,25 +2155,18 @@ func (node *Backup) doc(p *PrettyCfg) pretty.Doc { if node.Targets != nil { items = append(items, node.Targets.docRow(p)) } - if node.Nested { - if node.Subdir != nil { - items = append(items, p.row("INTO ", p.Doc(node.Subdir))) - items = append(items, p.row(" IN ", p.Doc(&node.To))) - } else if node.AppendToLatest { - items = append(items, p.row("INTO LATEST IN", p.Doc(&node.To))) - } else { - items = append(items, p.row("INTO", p.Doc(&node.To))) - } + if node.Subdir != nil { + items = append(items, p.row("INTO ", p.Doc(node.Subdir))) + items = append(items, p.row(" IN ", p.Doc(&node.To))) + } else if node.AppendToLatest { + items = append(items, p.row("INTO LATEST IN", p.Doc(&node.To))) } else { - items = append(items, p.row("TO", p.Doc(&node.To))) + items = append(items, p.row("INTO", p.Doc(&node.To))) } if node.AsOf.Expr != nil { items = append(items, node.AsOf.docRow(p)) } - if node.IncrementalFrom != nil { - items = append(items, p.row("INCREMENTAL FROM", p.Doc(&node.IncrementalFrom))) - } if !node.Options.IsDefault() { items = append(items, p.row("WITH", p.Doc(&node.Options))) } diff --git a/pkg/sql/sem/tree/walk.go b/pkg/sql/sem/tree/walk.go index 28fa6fecc02d..7ce4f116744f 100644 --- a/pkg/sql/sem/tree/walk.go +++ b/pkg/sql/sem/tree/walk.go @@ -1265,7 +1265,6 @@ func (n *DropTenant) walkStmt(v Visitor) Statement { // copyNode makes a copy of this Statement without recursing in any child Statements. func (stmt *Backup) copyNode() *Backup { stmtCopy := *stmt - stmtCopy.IncrementalFrom = append(Exprs(nil), stmt.IncrementalFrom...) return &stmtCopy } @@ -1290,15 +1289,7 @@ func (stmt *Backup) walkStmt(v Visitor) Statement { ret.To[i] = e } } - for i, expr := range stmt.IncrementalFrom { - e, changed := WalkExpr(v, expr) - if changed { - if ret == stmt { - ret = stmt.copyNode() - } - ret.IncrementalFrom[i] = e - } - } + if stmt.Options.EncryptionPassphrase != nil { pw, changed := WalkExpr(v, stmt.Options.EncryptionPassphrase) if changed {