Skip to content

Commit

Permalink
Revert "backupccl: disallow restore of backups older than the minimum…
Browse files Browse the repository at this point in the history
… supported binary version"

This reverts commit c39e5e4.

This change caused lots of unrelated roachtests to break, reverting
while we investigate those.

Release note: None

Epic: None
  • Loading branch information
adityamaru committed Mar 18, 2023
1 parent 0a72a49 commit c510676
Show file tree
Hide file tree
Showing 10 changed files with 356 additions and 221 deletions.
3 changes: 0 additions & 3 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -1427,7 +1427,6 @@ unreserved_keyword ::=
| 'UNKNOWN'
| 'UNLISTEN'
| 'UNLOGGED'
| 'UNSAFE_RESTORE_INCOMPATIBLE_VERSION'
| 'UNSET'
| 'UNSPLIT'
| 'UNTIL'
Expand Down Expand Up @@ -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 ) )*
Expand Down Expand Up @@ -3992,7 +3990,6 @@ bare_label_keywords ::=
| 'UNKNOWN'
| 'UNLISTEN'
| 'UNLOGGED'
| 'UNSAFE_RESTORE_INCOMPATIBLE_VERSION'
| 'UNSET'
| 'UNSPLIT'
| 'UNTIL'
Expand Down
104 changes: 42 additions & 62 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8445,86 +8445,66 @@ 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)
require.NoError(t, err)
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
Expand Down
15 changes: 3 additions & 12 deletions pkg/ccl/backupccl/restore_mid_schema_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
13 changes: 2 additions & 11 deletions pkg/ccl/backupccl/restore_old_sequences_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
23 changes: 2 additions & 21 deletions pkg/ccl/backupccl/restore_old_versions_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"},
Expand Down Expand Up @@ -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"},
Expand Down
71 changes: 17 additions & 54 deletions pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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 {
Expand Down
Loading

0 comments on commit c510676

Please sign in to comment.