Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
98805: kvserver: unskip merge queue during snapshot test r=nvanbenschoten a=AlexTalks

This change unskips `TestMergeQueueDoesNotInterruptReplicationChange`, which was skipped as potentially flaky in #94951. Despite repeated stress testing, this test no longer seems flaky, and correctly gets an error if attempting to enqueue a range in the merge queue during a replication change.

Fixes: #94951

Release note: None

98809: sql/logictest: clarify retry comment r=mgartner a=mgartner

Epic: None

Release note: None

98844: testutils: move tenant or server logic into test server r=stevendanna a=herkolategan

The logic to decide between returning a default test tenant, if one was started, or the server has been added to `testcluster` previously, but `testserver` can also benefit from exposing this logic for tests that do not require a whole cluster. Hence, this logic has been refactored to be available in both test cluster and test server.

Epic: CRDB-18499

98888: kvserver: remove duplicate logging in s-rebalancer r=andrewbaptist a=kvoli

Previously, the store rebalancer would log `load-based replica transfers successfully brought...` even when the store was not below the max threshold. The store rebalancer also logs when there are no available replica rebalancing actions remaining but the store still above the desired load threshold: `ran out of replicas worth transferring and load`. This log line was duplicated in both the post range rebalancing phase and in exiting the range rebalancing phase.

This commit stops duplication by removing the exit log line and ensures that the `load-based replica transfers successfully brought...` log line occurs only when actually successful.

Resolves: #98867

Release note: None

98959: Revert "Revert "backupccl: disallow restore of backups older than the minimum supported binary version"" r=dt a=adityamaru

This reverts commit c510676.

Some roachtests rely on fixtures generated before 22.2. To restore
such fixtures that are outside our compatibility window we must run
the restore with `unsafe_restore_incompatible_version`.

Fixes: #98918
Fixes: #98930
Release note: None
Epic: none

98997: multitenant: Increase timeout for kvtenantccl r=knz a=ajstorm

The new TestTenantUpgradeInterlock test takes a long time to run (up to a minute locally) and has started timing out in nightly runs. On the suspicion that it's timing out strictly because it's long running, increase the timeout to see if that resolves the failures.

Release note: None

99021: ui: fix jobs column name r=maryliag a=maryliag

The Last Execution Time called was previously labelled as Last Modified Time by mistake.
This commit fixes to use the correct name.

Epic: None

Release note (ui change): Update the Jobs table column name to the correct Last Execution Time.

Co-authored-by: Alex Sarkesian <sarkesian@cockroachlabs.com>
Co-authored-by: Marcus Gartner <marcus@cockroachlabs.com>
Co-authored-by: Herko Lategan <herko@cockroachlabs.com>
Co-authored-by: Austen McClernon <austen@cockroachlabs.com>
Co-authored-by: adityamaru <adityamaru@gmail.com>
Co-authored-by: Adam Storm <storm@cockroachlabs.com>
Co-authored-by: maryliag <marylia@cockroachlabs.com>
  • Loading branch information
8 people committed Mar 20, 2023
8 parents dd7766e + 68b7939 + 98d1add + 4df9c9b + 0b18c6e + d9b6fe7 + 95c90f9 + 8e24795 commit 5839fe0
Show file tree
Hide file tree
Showing 24 changed files with 267 additions and 377 deletions.
3 changes: 3 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -1427,6 +1427,7 @@ unreserved_keyword ::=
| 'UNKNOWN'
| 'UNLISTEN'
| 'UNLOGGED'
| 'UNSAFE_RESTORE_INCOMPATIBLE_VERSION'
| 'UNSET'
| 'UNSPLIT'
| 'UNTIL'
Expand Down Expand Up @@ -2664,6 +2665,7 @@ 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 @@ -3990,6 +3992,7 @@ bare_label_keywords ::=
| 'UNKNOWN'
| 'UNLISTEN'
| 'UNLOGGED'
| 'UNSAFE_RESTORE_INCOMPATIBLE_VERSION'
| 'UNSET'
| 'UNSPLIT'
| 'UNTIL'
Expand Down
104 changes: 62 additions & 42 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8445,66 +8445,86 @@ 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)
require.NoError(t, err)
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
Expand Down
15 changes: 12 additions & 3 deletions pkg/ccl/backupccl/restore_mid_schema_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
13 changes: 11 additions & 2 deletions pkg/ccl/backupccl/restore_old_sequences_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
23 changes: 21 additions & 2 deletions pkg/ccl/backupccl/restore_old_versions_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"},
Expand Down Expand Up @@ -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"},
Expand Down
71 changes: 54 additions & 17 deletions pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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 {
Expand Down
Loading

0 comments on commit 5839fe0

Please sign in to comment.