diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 07a142b5d5dd..18e6e45f02bb 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1427,7 +1427,6 @@ unreserved_keyword ::= | 'UNKNOWN' | 'UNLISTEN' | 'UNLOGGED' - | 'UNSAFE_RESTORE_INCOMPATIBLE_VERSION' | 'UNSET' | 'UNSPLIT' | 'UNTIL' @@ -2665,7 +2664,6 @@ restore_options ::= | 'TENANT' '=' string_or_placeholder | 'SCHEMA_ONLY' | 'VERIFY_BACKUP_TABLE_DATA' - | 'UNSAFE_RESTORE_INCOMPATIBLE_VERSION' scrub_option_list ::= ( scrub_option ) ( ( ',' scrub_option ) )* @@ -3992,7 +3990,6 @@ 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 25da98979905..9e88b2d86e6e 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -8445,21 +8445,20 @@ 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'`) } -// 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) { +func TestManifestTooNew(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - tc, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) + _, 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/cross_version'`) + 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, `DROP DATABASE r1`) // Load/deserialize the manifest so we can mess with it. - manifestPath := filepath.Join(rawDir, "cross_version", backupbase.BackupMetadataName) + manifestPath := filepath.Join(rawDir, "too_new", backupbase.BackupMetadataName) manifestData, err := os.ReadFile(manifestPath) require.NoError(t, err) manifestData, err = backupinfo.DecompressData(context.Background(), nil, manifestData) @@ -8467,64 +8466,45 @@ func TestRestoringAcrossVersions(t *testing.T) { var backupManifest backuppb.BackupManifest require.NoError(t, protoutil.Unmarshal(manifestData, &backupManifest)) - 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 */)) - } - - 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`) - }) - - 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}) + // 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 */)) - // 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'`) - }) + // 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-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`) - }) + // 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-nil-version-manifest", func(t *testing.T) { - // Nil out the version to match an old backup that lacked it. - setManifestClusterVersion(roachpb.Version{}) + // 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, "the backup is from a version older than our minimum restoreable 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`) } // 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 ba3afce459a8..c18fda7bf327 100644 --- a/pkg/ccl/backupccl/restore_mid_schema_change_test.go +++ b/pkg/ccl/backupccl/restore_mid_schema_change_test.go @@ -243,21 +243,12 @@ func restoreMidSchemaChange( require.NoError(t, err) sqlDB.Exec(t, "USE defaultdb") - // 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" + restoreQuery := "RESTORE defaultdb.* FROM LATEST IN $1" if isClusterRestore { - restoreQuery = "RESTORE FROM LATEST IN $1 WITH UNSAFE_RESTORE_INCOMPATIBLE_VERSION" + restoreQuery = "RESTORE FROM LATEST IN $1" } if isSchemaOnly { - restoreQuery = restoreQuery + ", schema_only" + restoreQuery = restoreQuery + "with 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 e1bd2a5c3bbc..fe3619b7aaab 100644 --- a/pkg/ccl/backupccl/restore_old_sequences_test.go +++ b/pkg/ccl/backupccl/restore_old_sequences_test.go @@ -79,18 +79,9 @@ func restoreOldSequencesTest(exportDir string, isSchemaOnly bool) func(t *testin sqlDB.Exec(t, `CREATE DATABASE test`) var unused string var importedRows int - // 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` + restoreQuery := `RESTORE test.* FROM LATEST IN $1` if isSchemaOnly { - restoreQuery = restoreQuery + ", schema_only" + restoreQuery = restoreQuery + " with 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 2d652f808627..406de2c936a1 100644 --- a/pkg/ccl/backupccl/restore_old_versions_test.go +++ b/pkg/ccl/backupccl/restore_old_versions_test.go @@ -110,17 +110,7 @@ func restoreOldVersionClusterTest(exportDir string) func(t *testing.T) { require.NoError(t, err) // Ensure that the restore succeeds. - // - // 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.Exec(t, `RESTORE FROM LATEST IN $1`, localFoo) sqlDB.CheckQueryResults(t, "SHOW DATABASES", [][]string{ {"data", "root", "NULL", "NULL", "{}", "NULL"}, @@ -282,16 +272,7 @@ func fullClusterRestoreSystemRoleMembersWithoutIDs(exportDir string) func(t *tes err := os.Symlink(exportDir, filepath.Join(tmpDir, "foo")) require.NoError(t, err) - // 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.Exec(t, fmt.Sprintf("RESTORE FROM '%s'", 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 e5a5a9e0d85b..7f7176a34ac2 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -882,15 +882,14 @@ 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, - UnsafeRestoreIncompatibleVersion: opts.UnsafeRestoreIncompatibleVersion, + SkipMissingFKs: opts.SkipMissingFKs, + SkipMissingSequences: opts.SkipMissingSequences, + SkipMissingSequenceOwners: opts.SkipMissingSequenceOwners, + SkipMissingViews: opts.SkipMissingViews, + SkipMissingUDFs: opts.SkipMissingUDFs, + Detached: opts.Detached, + SchemaOnly: opts.SchemaOnly, + VerifyData: opts.VerifyData, } if opts.EncryptionPassphrase != nil { @@ -1418,48 +1417,6 @@ 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, @@ -1635,9 +1592,15 @@ func doRestorePlan( }() currentVersion := p.ExecCfg().Settings.Version.ActiveVersion(ctx) - if err := checkBackupManifestVersionCompatability(p, currentVersion, mainBackupManifests, - restoreStmt.Options.UnsafeRestoreIncompatibleVersion); err != nil { - return err + 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 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 863cb823ad87..c8d012cd00c5 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports +++ b/pkg/ccl/backupccl/testdata/backup-restore/in-progress-imports @@ -314,3 +314,273 @@ 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 af3b58a1b705..53292076dc34 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -987,7 +987,7 @@ func (u *sqlSymUnion) showCreateFormatOption() tree.ShowCreateFormatOption { %token TRUNCATE TRUSTED TYPE TYPES %token TRACING -%token UNBOUNDED UNCOMMITTED UNION UNIQUE UNKNOWN UNLISTEN UNLOGGED UNSAFE_RESTORE_INCOMPATIBLE_VERSION UNSPLIT +%token UNBOUNDED UNCOMMITTED UNION UNIQUE UNKNOWN UNLISTEN UNLOGGED 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 @@ -3769,11 +3769,6 @@ restore_options: { $$.val = &tree.RestoreOptions{VerifyData: true} } -| UNSAFE_RESTORE_INCOMPATIBLE_VERSION - { - $$.val = &tree.RestoreOptions{UnsafeRestoreIncompatibleVersion: true} - } - import_format: name { @@ -16604,7 +16599,6 @@ unreserved_keyword: | UNKNOWN | UNLISTEN | UNLOGGED -| UNSAFE_RESTORE_INCOMPATIBLE_VERSION | UNSET | UNSPLIT | UNTIL @@ -17168,7 +17162,6 @@ 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 6c10b307df1d..b3edc4b681fc 100644 --- a/pkg/sql/parser/testdata/backup_restore +++ b/pkg/sql/parser/testdata/backup_restore @@ -823,14 +823,6 @@ 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' ---- @@ -938,14 +930,6 @@ 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 8fc4ce5d9a46..4cf0ee43d14d 100644 --- a/pkg/sql/sem/tree/backup.go +++ b/pkg/sql/sem/tree/backup.go @@ -126,25 +126,24 @@ 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 - UnsafeRestoreIncompatibleVersion 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 } var _ NodeFormatter = &RestoreOptions{} @@ -487,11 +486,6 @@ 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. @@ -624,14 +618,6 @@ 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 } @@ -655,8 +641,7 @@ func (o RestoreOptions) IsDefault() bool { o.ForceTenantID == options.ForceTenantID && o.SchemaOnly == options.SchemaOnly && o.VerifyData == options.VerifyData && - o.IncludeAllSecondaryTenants == options.IncludeAllSecondaryTenants && - o.UnsafeRestoreIncompatibleVersion == options.UnsafeRestoreIncompatibleVersion + o.IncludeAllSecondaryTenants == options.IncludeAllSecondaryTenants } // BackupTargetList represents a list of targets.