From c39e5e4cb11e22e5269be173487d17f820f8b6ea Mon Sep 17 00:00:00 2001 From: adityamaru Date: Tue, 14 Mar 2023 13:05:24 -0400 Subject: [PATCH] backupccl: disallow restore of backups older than the minimum supported binary version As outlined in https://www.cockroachlabs.com/docs/v22.2/restoring-backups-across-versions.html, we do not support restoring backups outside our versioning policy window. This patch enforces the policy by ensuring that the manifest's version is greater than equal to the minimum supported binary version that the current binary can interoperate with. Additionally, this patch introduces an `UNSAFE_RESTORE_INCOMPATIBLE_VERSION` option that can be used to skip compatability checks and forcefully restore backups taken outside our compatability window. This has primarily been added to allow development branches to restore backups taken on release branches, but can be used as a break glass option where restoring a backup is absolutely necessary. No guarantees are made about the correctness of the restore when using this option. Release note (sql change): Disallow the restore of backups taken on a cluster version that is older than the minimum binary version the current cluster can interoperate with. This will be described in an updated version of the policy outlined in https://www.cockroachlabs.com/docs/v22.2/restoring-backups-across-versions.html. Fixes: #94295 --- docs/generated/sql/bnf/stmt_block.bnf | 3 + pkg/ccl/backupccl/backup_test.go | 104 ++++--- .../restore_mid_schema_change_test.go | 15 +- .../backupccl/restore_old_sequences_test.go | 13 +- .../backupccl/restore_old_versions_test.go | 23 +- pkg/ccl/backupccl/restore_planning.go | 71 +++-- .../backup-restore/in-progress-imports | 270 ------------------ pkg/sql/parser/sql.y | 9 +- pkg/sql/parser/testdata/backup_restore | 16 ++ pkg/sql/sem/tree/backup.go | 53 ++-- 10 files changed, 221 insertions(+), 356 deletions(-) diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index ec6122c294c0..4464cff5a438 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1426,6 +1426,7 @@ unreserved_keyword ::= | 'UNKNOWN' | 'UNLISTEN' | 'UNLOGGED' + | 'UNSAFE_RESTORE_INCOMPATIBLE_VERSION' | 'UNSET' | 'UNSPLIT' | 'UNTIL' @@ -2660,6 +2661,7 @@ restore_options ::= | 'TENANT' '=' string_or_placeholder | 'SCHEMA_ONLY' | 'VERIFY_BACKUP_TABLE_DATA' + | 'UNSAFE_RESTORE_INCOMPATIBLE_VERSION' scrub_option_list ::= ( scrub_option ) ( ( ',' scrub_option ) )* @@ -3985,6 +3987,7 @@ bare_label_keywords ::= | 'UNKNOWN' | 'UNLISTEN' | 'UNLOGGED' + | 'UNSAFE_RESTORE_INCOMPATIBLE_VERSION' | 'UNSET' | 'UNSPLIT' | 'UNTIL' diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index fbf9b1f38c65..a2e671a8a848 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -8437,20 +8437,21 @@ func TestIncorrectAccessOfFilesInBackupMetadata(t *testing.T) { sqlDB.ExpectErr(t, "assertion: this placeholder legacy Descriptor entry should never be used", `RESTORE DATABASE r1 FROM LATEST IN 'nodelocal://0/test' WITH new_db_name = 'r2'`) } -func TestManifestTooNew(t *testing.T) { +// TestRestoringAcrossVersions test that users are only allowed to restore +// backups taken on a version >= the minimum supported binary version of the +// current active cluster version. +func TestRestoringAcrossVersions(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - _, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) + tc, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) defer cleanupFn() + sqlDB.Exec(t, `CREATE DATABASE r1`) - sqlDB.Exec(t, `BACKUP DATABASE r1 TO 'nodelocal://0/too_new'`) - sqlDB.Exec(t, `DROP DATABASE r1`) - // Prove we can restore. - sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) + sqlDB.Exec(t, `BACKUP DATABASE r1 TO 'nodelocal://0/cross_version'`) sqlDB.Exec(t, `DROP DATABASE r1`) // Load/deserialize the manifest so we can mess with it. - manifestPath := filepath.Join(rawDir, "too_new", backupbase.BackupMetadataName) + manifestPath := filepath.Join(rawDir, "cross_version", backupbase.BackupMetadataName) manifestData, err := os.ReadFile(manifestPath) require.NoError(t, err) manifestData, err = backupinfo.DecompressData(context.Background(), nil, manifestData) @@ -8458,45 +8459,64 @@ func TestManifestTooNew(t *testing.T) { var backupManifest backuppb.BackupManifest require.NoError(t, protoutil.Unmarshal(manifestData, &backupManifest)) - // Bump the version and write it back out to make it look newer. - backupManifest.ClusterVersion = roachpb.Version{Major: math.MaxInt32, Minor: 1} - manifestData, err = protoutil.Marshal(&backupManifest) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) - // Also write the checksum file to match the new manifest. - checksum, err := backupinfo.GetChecksum(manifestData) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) + setManifestClusterVersion := func(version roachpb.Version) { + backupManifest.ClusterVersion = version + manifestData, err = protoutil.Marshal(&backupManifest) + require.NoError(t, err) + require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) + // Also write the checksum file to match the new manifest. + checksum, err := backupinfo.GetChecksum(manifestData) + require.NoError(t, err) + require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) + } - // Verify we reject it. - sqlDB.ExpectErr(t, "backup from version 2147483647.1 is newer than current version", `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) + t.Run("restore-same-version", func(t *testing.T) { + // Prove we can restore a backup taken on our current version. + sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/cross_version'`) + sqlDB.Exec(t, `DROP DATABASE r1`) + }) - // Bump the version down and write it back out to make it look older. - backupManifest.ClusterVersion = roachpb.Version{Major: 20, Minor: 2, Internal: 2} - manifestData, err = protoutil.Marshal(&backupManifest) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) - // Also write the checksum file to match the new manifest. - checksum, err = backupinfo.GetChecksum(manifestData) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) + t.Run("restore-newer-version", func(t *testing.T) { + // Bump the version and write it back out to make it look newer. + setManifestClusterVersion(roachpb.Version{Major: math.MaxInt32, Minor: 1}) - // Prove we can restore again. - sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) - sqlDB.Exec(t, `DROP DATABASE r1`) + // Verify we reject it. + sqlDB.ExpectErr(t, "backup from version 2147483647.1 is newer than current version", + `RESTORE DATABASE r1 FROM 'nodelocal://0/cross_version'`) + }) - // Nil out the version to match an old backup that lacked it. - backupManifest.ClusterVersion = roachpb.Version{} - manifestData, err = protoutil.Marshal(&backupManifest) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) - // Also write the checksum file to match the new manifest. - checksum, err = backupinfo.GetChecksum(manifestData) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) - // Prove we can restore again. - sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) - sqlDB.Exec(t, `DROP DATABASE r1`) + t.Run("restore-older-major-version", func(t *testing.T) { + // Bump the version down to outside our MinBinarySupportedVersion, and write + // it back out. This makes it ineligible for restore because of our restore + // version policy. + minSupportedVersion := tc.Server(0).ClusterSettings().Version.BinaryMinSupportedVersion() + minSupportedVersion.Major -= 1 + setManifestClusterVersion(minSupportedVersion) + + // Verify we reject it. + sqlDB.ExpectErr(t, + fmt.Sprintf("backup from version %s is older than the minimum restoreable version", minSupportedVersion.String()), + `RESTORE DATABASE r1 FROM 'nodelocal://0/cross_version'`) + }) + + t.Run("restore-min-binary-version", func(t *testing.T) { + // Bump the version down to the min supported binary version, and write it + // back out. This makes it eligible for restore because of our restore + // version policy. + minBinaryVersion := tc.Server(0).ClusterSettings().Version.BinaryMinSupportedVersion() + setManifestClusterVersion(minBinaryVersion) + sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/cross_version'`) + sqlDB.Exec(t, `DROP DATABASE r1`) + }) + + t.Run("restore-nil-version-manifest", func(t *testing.T) { + // Nil out the version to match an old backup that lacked it. + setManifestClusterVersion(roachpb.Version{}) + + // Verify we reject it. + sqlDB.ExpectErr(t, "the backup is from a version older than our minimum restoreable version", + `RESTORE DATABASE r1 FROM 'nodelocal://0/cross_version'`) + }) } // TestManifestBitFlip tests that we can detect a corrupt manifest when a bit diff --git a/pkg/ccl/backupccl/restore_mid_schema_change_test.go b/pkg/ccl/backupccl/restore_mid_schema_change_test.go index c18fda7bf327..ba3afce459a8 100644 --- a/pkg/ccl/backupccl/restore_mid_schema_change_test.go +++ b/pkg/ccl/backupccl/restore_mid_schema_change_test.go @@ -243,12 +243,21 @@ func restoreMidSchemaChange( require.NoError(t, err) sqlDB.Exec(t, "USE defaultdb") - restoreQuery := "RESTORE defaultdb.* FROM LATEST IN $1" + // The restore queries are run with `UNSAFE_RESTORE_INCOMPATIBLE_VERSION` + // option to ensure the restore is successful on development branches. This + // is because, while the backups were generated on release branches and have + // versions such as 22.2 in their manifest, the development branch will have + // a BinaryMinSupportedVersion offset by the clusterversion.DevOffset + // described in `pkg/clusterversion/cockroach_versions.go`. This will mean + // that the manifest version is always less than the + // BinaryMinSupportedVersion which will in turn fail the restore unless we + // pass in the specified option to elide the compatability check. + restoreQuery := "RESTORE defaultdb.* FROM LATEST IN $1 WITH UNSAFE_RESTORE_INCOMPATIBLE_VERSION" if isClusterRestore { - restoreQuery = "RESTORE FROM LATEST IN $1" + restoreQuery = "RESTORE FROM LATEST IN $1 WITH UNSAFE_RESTORE_INCOMPATIBLE_VERSION" } if isSchemaOnly { - restoreQuery = restoreQuery + "with schema_only" + restoreQuery = restoreQuery + ", schema_only" } log.Infof(context.Background(), "%+v", sqlDB.QueryStr(t, "SHOW BACKUP LATEST IN $1", localFoo)) sqlDB.Exec(t, restoreQuery, localFoo) diff --git a/pkg/ccl/backupccl/restore_old_sequences_test.go b/pkg/ccl/backupccl/restore_old_sequences_test.go index fe3619b7aaab..e1bd2a5c3bbc 100644 --- a/pkg/ccl/backupccl/restore_old_sequences_test.go +++ b/pkg/ccl/backupccl/restore_old_sequences_test.go @@ -79,9 +79,18 @@ func restoreOldSequencesTest(exportDir string, isSchemaOnly bool) func(t *testin sqlDB.Exec(t, `CREATE DATABASE test`) var unused string var importedRows int - restoreQuery := `RESTORE test.* FROM LATEST IN $1` + // The restore queries are run with `UNSAFE_RESTORE_INCOMPATIBLE_VERSION` + // option to ensure the restore is successful on development branches. This + // is because, while the backups were generated on release branches and have + // versions such as 22.2 in their manifest, the development branch will have + // a BinaryMinSupportedVersion offset by the clusterversion.DevOffset + // described in `pkg/clusterversion/cockroach_versions.go`. This will mean + // that the manifest version is always less than the + // BinaryMinSupportedVersion which will in turn fail the restore unless we + // pass in the specified option to elide the compatability check. + restoreQuery := `RESTORE test.* FROM LATEST IN $1 WITH UNSAFE_RESTORE_INCOMPATIBLE_VERSION` if isSchemaOnly { - restoreQuery = restoreQuery + " with schema_only" + restoreQuery = restoreQuery + ", schema_only" } sqlDB.QueryRow(t, restoreQuery, localFoo).Scan( &unused, &unused, &unused, &importedRows, &unused, &unused, diff --git a/pkg/ccl/backupccl/restore_old_versions_test.go b/pkg/ccl/backupccl/restore_old_versions_test.go index 406de2c936a1..2d652f808627 100644 --- a/pkg/ccl/backupccl/restore_old_versions_test.go +++ b/pkg/ccl/backupccl/restore_old_versions_test.go @@ -110,7 +110,17 @@ func restoreOldVersionClusterTest(exportDir string) func(t *testing.T) { require.NoError(t, err) // Ensure that the restore succeeds. - sqlDB.Exec(t, `RESTORE FROM LATEST IN $1`, localFoo) + // + // The restore queries are run with `UNSAFE_RESTORE_INCOMPATIBLE_VERSION` + // option to ensure the restore is successful on development branches. This + // is because, while the backups were generated on release branches and have + // versions such as 22.2 in their manifest, the development branch will have + // a BinaryMinSupportedVersion offset by the clusterversion.DevOffset + // described in `pkg/clusterversion/cockroach_versions.go`. This will mean + // that the manifest version is always less than the + // BinaryMinSupportedVersion which will in turn fail the restore unless we + // pass in the specified option to elide the compatability check. + sqlDB.Exec(t, `RESTORE FROM LATEST IN $1 WITH UNSAFE_RESTORE_INCOMPATIBLE_VERSION`, localFoo) sqlDB.CheckQueryResults(t, "SHOW DATABASES", [][]string{ {"data", "root", "NULL", "NULL", "{}", "NULL"}, @@ -272,7 +282,16 @@ func fullClusterRestoreSystemRoleMembersWithoutIDs(exportDir string) func(t *tes err := os.Symlink(exportDir, filepath.Join(tmpDir, "foo")) require.NoError(t, err) - sqlDB.Exec(t, fmt.Sprintf("RESTORE FROM '%s'", localFoo)) + // The restore queries are run with `UNSAFE_RESTORE_INCOMPATIBLE_VERSION` + // option to ensure the restore is successful on development branches. This + // is because, while the backups were generated on release branches and have + // versions such as 22.2 in their manifest, the development branch will have + // a BinaryMinSupportedVersion offset by the clusterversion.DevOffset + // described in `pkg/clusterversion/cockroach_versions.go`. This will mean + // that the manifest version is always less than the + // BinaryMinSupportedVersion which will in turn fail the restore unless we + // pass in the specified option to elide the compatability check. + sqlDB.Exec(t, fmt.Sprintf("RESTORE FROM '%s' WITH UNSAFE_RESTORE_INCOMPATIBLE_VERSION", localFoo)) sqlDB.CheckQueryResults(t, "SELECT * FROM system.role_members", [][]string{ {"admin", "root", "true", "2", "1"}, diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 7f7176a34ac2..e5a5a9e0d85b 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -882,14 +882,15 @@ func resolveOptionsForRestoreJobDescription( } newOpts := tree.RestoreOptions{ - SkipMissingFKs: opts.SkipMissingFKs, - SkipMissingSequences: opts.SkipMissingSequences, - SkipMissingSequenceOwners: opts.SkipMissingSequenceOwners, - SkipMissingViews: opts.SkipMissingViews, - SkipMissingUDFs: opts.SkipMissingUDFs, - Detached: opts.Detached, - SchemaOnly: opts.SchemaOnly, - VerifyData: opts.VerifyData, + SkipMissingFKs: opts.SkipMissingFKs, + SkipMissingSequences: opts.SkipMissingSequences, + SkipMissingSequenceOwners: opts.SkipMissingSequenceOwners, + SkipMissingViews: opts.SkipMissingViews, + SkipMissingUDFs: opts.SkipMissingUDFs, + Detached: opts.Detached, + SchemaOnly: opts.SchemaOnly, + VerifyData: opts.VerifyData, + UnsafeRestoreIncompatibleVersion: opts.UnsafeRestoreIncompatibleVersion, } if opts.EncryptionPassphrase != nil { @@ -1417,6 +1418,48 @@ func checkClusterRegions( return nil } +// checkBackupManifestVersionCompatability performs various checks to ensure +// that the manifests we are about to restore are from backups taken on a +// version compatible with our current version. +func checkBackupManifestVersionCompatability( + p sql.PlanHookState, + currentActiveVersion clusterversion.ClusterVersion, + mainBackupManifests []backuppb.BackupManifest, + unsafeRestoreIncompatibleVersion bool, +) error { + // We support restoring a backup that was taken on a cluster with a cluster + // version >= the earliest binary version that we can interoperate with. + minimumRestoreableVersion := p.ExecCfg().Settings.Version.BinaryMinSupportedVersion() + for i := range mainBackupManifests { + v := mainBackupManifests[i].ClusterVersion + // This is the "cluster" version that does not change between patch releases + // but rather just tracks migrations run. If the backup is more migrated + // than this cluster, then this cluster isn't ready to restore this backup. + if currentActiveVersion.Less(v) { + return errors.Errorf("backup from version %s is newer than current version %s", v, currentActiveVersion) + } + + // If the backup is from a version earlier than the minimum restoreable + // version, then we do not support restoring it. Unless, the user has + // explicitly run the restore with the `UNSAFE_RESTORE_INCOMPATIBLE_VERSION` + // option. + if !unsafeRestoreIncompatibleVersion && v.Less(minimumRestoreableVersion) { + if v.Major == 0 { + // This accounts for manifests that were generated on a version before + // the `ClusterVersion` field exists. + return errors.WithHint(errors.Newf("the backup is from a version older than our "+ + "minimum restoreable version %s", minimumRestoreableVersion), + "refer to our documentation about restoring across versions: https://www.cockroachlabs.com/docs/v22.2/restoring-backups-across-versions.html") + } + return errors.WithHint(errors.Newf("backup from version %s is older than the "+ + "minimum restoreable version %s", v, minimumRestoreableVersion), + "refer to our documentation about restoring across versions: https://www.cockroachlabs.com/docs/v22.2/restoring-backups-across-versions.html") + } + } + + return nil +} + func doRestorePlan( ctx context.Context, restoreStmt *tree.Restore, @@ -1592,15 +1635,9 @@ func doRestorePlan( }() currentVersion := p.ExecCfg().Settings.Version.ActiveVersion(ctx) - for i := range mainBackupManifests { - if v := mainBackupManifests[i].ClusterVersion; v.Major != 0 { - // This is the "cluster" version that does not change between patches but - // rather just tracks migrations run. If the backup is more migrated than - // this cluster, then this cluster isn't ready to restore this backup. - if currentVersion.Less(v) { - return errors.Errorf("backup from version %s is newer than current version %s", v, currentVersion) - } - } + if err := checkBackupManifestVersionCompatability(p, currentVersion, mainBackupManifests, + restoreStmt.Options.UnsafeRestoreIncompatibleVersion); err != nil { + return err } if restoreStmt.DescriptorCoverage == tree.AllDescriptors { diff --git a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports index c8d012cd00c5..863cb823ad87 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports +++ b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports @@ -314,273 +314,3 @@ SELECT * FROM d.foofoo; ---- 1 x 10 x0 - - -####################### -# Version Gate Testing -####################### - -# In an unfinalized cluster, back up some in-progress imports, and ensure that once the tables come -# back online, we fully back them again, even if the cluster is not fully upgraded. Test on cluster -# and database backups. -# -# Note that during IMPORT planning on an unfinalized cluster, the -# ImportStartTime is not bound to the table's descriptor, therefore during -# RESTORE AOST in-progress IMPORT, these tables should get thrown out. -# -# TODO(msbutler): cover mixed version RESTORE TABLE -# -# Disabled to run within tenants because version gating infra does not play nice within tenants. -# More investigation required. - - -new-cluster name=s4 share-io-dir=s1 allow-implicit-access beforeVersion=Start22_2 disable-tenant ----- - -exec-sql -CREATE DATABASE d; -USE d; -CREATE TABLE foo (i INT PRIMARY KEY, s STRING); -CREATE TABLE foofoo (i INT PRIMARY KEY, s STRING); -INSERT INTO foofoo VALUES (10, 'x0'); -CREATE TABLE baz (i INT PRIMARY KEY, s STRING); -INSERT INTO baz VALUES (1, 'x'),(2,'y'),(3,'z'); ----- - - -exec-sql -SET CLUSTER SETTING jobs.debug.pausepoints = 'import.after_ingest'; ----- - - -exec-sql -EXPORT INTO CSV 'nodelocal://0/export1/' FROM SELECT * FROM baz WHERE i = 1; ----- - - -# Pause the import job, in order to back up the importing data. -import expect-pausepoint tag=b -IMPORT INTO foo (i,s) CSV DATA ('nodelocal://0/export1/export*-n*.0.csv') ----- -job paused at pausepoint - - -import expect-pausepoint tag=bb -IMPORT INTO foofoo (i,s) CSV DATA ('nodelocal://0/export1/export*-n*.0.csv') ----- -job paused at pausepoint - - -# The first backup in the chain will capture data from offline tables, even -# though the cluster has not finalized to 22.2. Ensure this works in cluster and database backups. -# The 'database' and 'database_upgrade' backup chains will test different backup chain / upgrade -# sequences. - -exec-sql -BACKUP INTO 'nodelocal://0/cluster/' with revision_history; ----- - - -exec-sql -BACKUP DATABASE d INTO 'nodelocal://0/database/' with revision_history; ----- - - -exec-sql -BACKUP DATABASE d INTO 'nodelocal://0/database_upgrade/' with revision_history; ----- - - -save-cluster-ts tag=m0 ----- - - -# This next set of incremental backups should not capture any new data, as no new data was ingested -# into the table since the last backup. -exec-sql -BACKUP INTO LATEST IN 'nodelocal://0/cluster/' with revision_history; ----- - - -exec-sql -BACKUP DATABASE d INTO LATEST IN 'nodelocal://0/database/' with revision_history; ----- - - -exec-sql -CREATE VIEW show_cluster_backup AS -SELECT - database_name, object_name, object_type, rows, backup_type -FROM - [SHOW BACKUP FROM LATEST IN 'nodelocal://0/cluster/'] -WHERE - object_name = 'foo' or object_name = 'foofoo' -ORDER BY - start_time, database_name; ----- - - -exec-sql -CREATE VIEW show_database_backup AS -SELECT - database_name, object_name, object_type, rows, backup_type -FROM - [SHOW BACKUP FROM LATEST IN 'nodelocal://0/database/'] -WHERE - object_name = 'foo' or object_name = 'foofoo' -ORDER BY - start_time, database_name; ----- - - -query-sql -SELECT * FROM show_cluster_backup; ----- -d foo table 1 full -d foofoo table 2 full -d foo table 0 incremental -d foofoo table 0 incremental - - -query-sql -SELECT * FROM show_database_backup; ----- -d foo table 1 full -d foofoo table 2 full -d foo table 0 incremental -d foofoo table 0 incremental - - -exec-sql -SET CLUSTER SETTING jobs.debug.pausepoints = ''; ----- - - -# Resume the job so the next set of incremental backups observes that tables are back online -job resume=b ----- - - -job tag=b wait-for-state=succeeded ----- - - -job resume=bb ----- - - -job tag=bb wait-for-state=succeeded ----- - - -# Ensure that once the tables come back online, everything gets backed -# up again, as these imports may have non-mvcc ops in them. Ensure this in the -# unfinalized cluster and in the finalized cluster. -exec-sql -BACKUP INTO LATEST IN 'nodelocal://0/cluster/' with revision_history; ----- - - -exec-sql -BACKUP DATABASE d INTO LATEST IN 'nodelocal://0/database/' with revision_history; ----- - - -query-sql -SELECT * FROM show_cluster_backup; ----- -d foo table 1 full -d foofoo table 2 full -d foo table 0 incremental -d foofoo table 0 incremental -d foo table 2 incremental -d foofoo table 3 incremental - - -query-sql -SELECT * FROM show_database_backup; ----- -d foo table 1 full -d foofoo table 2 full -d foo table 0 incremental -d foofoo table 0 incremental -d foo table 2 incremental -d foofoo table 3 incremental - - -upgrade-cluster version=Start22_2 ----- - -exec-sql -BACKUP DATABASE d INTO LATEST IN 'nodelocal://0/database_upgrade/' with revision_history; ----- - -query-sql -SELECT - database_name, object_name, object_type, rows, backup_type -FROM - [SHOW BACKUP FROM LATEST IN 'nodelocal://0/database_upgrade/'] -WHERE - object_name = 'foo' or object_name = 'foofoo' -ORDER BY - start_time, database_name; ----- -d foo table 1 full -d foofoo table 2 full -d foo table 2 incremental -d foofoo table 3 incremental - - -# Restore the backups taken from a mixed version chain -new-cluster name=s5 share-io-dir=s1 allow-implicit-access disable-tenant ----- - - -# Ensure the RESTOREs omit the tables with in progress imports (foo and foofoo) -# as their descriptors will not have the import start time. -restore aost=m0 -RESTORE FROM LATEST IN 'nodelocal://0/cluster/' AS OF SYSTEM TIME m0; ----- - - -query-sql -SELECT table_name FROM [SHOW TABLES FROM d]; ----- -baz - - -exec-sql -DROP DATABASE d; ----- - - -restore aost=m0 -RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/database/' AS OF SYSTEM TIME m0; ----- - - -query-sql -SELECT table_name FROM [SHOW TABLES FROM d]; ----- -baz - - -exec-sql -DROP DATABASE d; ----- - - -# Restore AOST after the table comes back online -restore -RESTORE DATABASE d FROM LATEST IN 'nodelocal://0/database/'; ----- - - -query-sql -SELECT table_name FROM [SHOW TABLES FROM d]; ----- -foo -foofoo -baz -show_cluster_backup -show_database_backup diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 9038ada803b3..141c03484a23 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -984,7 +984,7 @@ func (u *sqlSymUnion) showTenantOpts() tree.ShowTenantOptions { %token TRUNCATE TRUSTED TYPE TYPES %token TRACING -%token UNBOUNDED UNCOMMITTED UNION UNIQUE UNKNOWN UNLISTEN UNLOGGED UNSPLIT +%token UNBOUNDED UNCOMMITTED UNION UNIQUE UNKNOWN UNLISTEN UNLOGGED UNSAFE_RESTORE_INCOMPATIBLE_VERSION UNSPLIT %token UPDATE UPSERT UNSET UNTIL USE USER USERS USING UUID %token VALID VALIDATE VALUE VALUES VARBIT VARCHAR VARIADIC VERIFY_BACKUP_TABLE_DATA VIEW VARYING VIEWACTIVITY VIEWACTIVITYREDACTED VIEWDEBUG @@ -3765,6 +3765,11 @@ restore_options: { $$.val = &tree.RestoreOptions{VerifyData: true} } +| UNSAFE_RESTORE_INCOMPATIBLE_VERSION + { + $$.val = &tree.RestoreOptions{UnsafeRestoreIncompatibleVersion: true} + } + import_format: name { @@ -16574,6 +16579,7 @@ unreserved_keyword: | UNKNOWN | UNLISTEN | UNLOGGED +| UNSAFE_RESTORE_INCOMPATIBLE_VERSION | UNSET | UNSPLIT | UNTIL @@ -17136,6 +17142,7 @@ bare_label_keywords: | UNKNOWN | UNLISTEN | UNLOGGED +| UNSAFE_RESTORE_INCOMPATIBLE_VERSION | UNSET | UNSPLIT | UNTIL diff --git a/pkg/sql/parser/testdata/backup_restore b/pkg/sql/parser/testdata/backup_restore index b3edc4b681fc..6c10b307df1d 100644 --- a/pkg/sql/parser/testdata/backup_restore +++ b/pkg/sql/parser/testdata/backup_restore @@ -823,6 +823,14 @@ RESTORE FROM ('latest') IN ('bar') WITH detached, include_all_secondary_tenants RESTORE FROM '_' IN '_' WITH detached, include_all_secondary_tenants = _ -- literals removed RESTORE FROM 'latest' IN 'bar' WITH detached, include_all_secondary_tenants = true -- identifiers removed +parse +RESTORE FROM LATEST IN 'bar' WITH unsafe_restore_incompatible_version, detached +---- +RESTORE FROM 'latest' IN 'bar' WITH detached, unsafe_restore_incompatible_version -- normalized! +RESTORE FROM ('latest') IN ('bar') WITH detached, unsafe_restore_incompatible_version -- fully parenthesized +RESTORE FROM '_' IN '_' WITH detached, unsafe_restore_incompatible_version -- literals removed +RESTORE FROM 'latest' IN 'bar' WITH detached, unsafe_restore_incompatible_version -- identifiers removed + error BACKUP foo TO 'bar' WITH key1, key2 = 'value' ---- @@ -930,6 +938,14 @@ DETAIL: source SQL: RESTORE FROM 'bar' WITH include_all_secondary_tenants=false, include_all_secondary_tenants ^ +error +RESTORE FROM 'bar' WITH unsafe_restore_incompatible_version, unsafe_restore_incompatible_version +---- +at or near "unsafe_restore_incompatible_version": syntax error: unsafe_restore_incompatible_version specified multiple times +DETAIL: source SQL: +RESTORE FROM 'bar' WITH unsafe_restore_incompatible_version, unsafe_restore_incompatible_version + ^ + error BACKUP ROLE foo, bar TO 'baz' ---- diff --git a/pkg/sql/sem/tree/backup.go b/pkg/sql/sem/tree/backup.go index 4cf0ee43d14d..8fc4ce5d9a46 100644 --- a/pkg/sql/sem/tree/backup.go +++ b/pkg/sql/sem/tree/backup.go @@ -126,24 +126,25 @@ func (node Backup) Coverage() DescriptorCoverage { // RestoreOptions describes options for the RESTORE execution. type RestoreOptions struct { - EncryptionPassphrase Expr - DecryptionKMSURI StringOrPlaceholderOptList - IntoDB Expr - SkipMissingFKs bool - SkipMissingSequences bool - SkipMissingSequenceOwners bool - SkipMissingViews bool - SkipMissingUDFs bool - Detached bool - SkipLocalitiesCheck bool - DebugPauseOn Expr - NewDBName Expr - IncludeAllSecondaryTenants Expr - IncrementalStorage StringOrPlaceholderOptList - AsTenant Expr - ForceTenantID Expr - SchemaOnly bool - VerifyData bool + EncryptionPassphrase Expr + DecryptionKMSURI StringOrPlaceholderOptList + IntoDB Expr + SkipMissingFKs bool + SkipMissingSequences bool + SkipMissingSequenceOwners bool + SkipMissingViews bool + SkipMissingUDFs bool + Detached bool + SkipLocalitiesCheck bool + DebugPauseOn Expr + NewDBName Expr + IncludeAllSecondaryTenants Expr + IncrementalStorage StringOrPlaceholderOptList + AsTenant Expr + ForceTenantID Expr + SchemaOnly bool + VerifyData bool + UnsafeRestoreIncompatibleVersion bool } var _ NodeFormatter = &RestoreOptions{} @@ -486,6 +487,11 @@ func (o *RestoreOptions) Format(ctx *FmtCtx) { maybeAddSep() ctx.WriteString("verify_backup_table_data") } + + if o.UnsafeRestoreIncompatibleVersion { + maybeAddSep() + ctx.WriteString("unsafe_restore_incompatible_version") + } } // CombineWith merges other backup options into this backup options struct. @@ -618,6 +624,14 @@ func (o *RestoreOptions) CombineWith(other *RestoreOptions) error { o.IncludeAllSecondaryTenants = other.IncludeAllSecondaryTenants } + if o.UnsafeRestoreIncompatibleVersion { + if other.UnsafeRestoreIncompatibleVersion { + return errors.New("unsafe_restore_incompatible_version specified multiple times") + } + } else { + o.UnsafeRestoreIncompatibleVersion = other.UnsafeRestoreIncompatibleVersion + } + return nil } @@ -641,7 +655,8 @@ func (o RestoreOptions) IsDefault() bool { o.ForceTenantID == options.ForceTenantID && o.SchemaOnly == options.SchemaOnly && o.VerifyData == options.VerifyData && - o.IncludeAllSecondaryTenants == options.IncludeAllSecondaryTenants + o.IncludeAllSecondaryTenants == options.IncludeAllSecondaryTenants && + o.UnsafeRestoreIncompatibleVersion == options.UnsafeRestoreIncompatibleVersion } // BackupTargetList represents a list of targets.