From 68b7939ab7b6191420056138fc4b75a70a76e994 Mon Sep 17 00:00:00 2001 From: Alex Sarkesian Date: Thu, 16 Mar 2023 15:29:38 -0400 Subject: [PATCH 1/8] kvserver: unskip merge queue during snapshot test 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 --- pkg/kv/kvserver/replica_learner_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index ebb248424b91..08f3b4252317 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -1721,7 +1721,6 @@ func TestLearnerAndJointConfigAdminMerge(t *testing.T) { func TestMergeQueueDoesNotInterruptReplicationChange(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - skip.WithIssue(t, 94951) ctx := context.Background() var activateSnapshotTestingKnob int64 blockSnapshot := make(chan struct{}) From 98d1addd4eefac9fd022dc7db7bb2ed67371fcd3 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Thu, 16 Mar 2023 16:12:30 -0400 Subject: [PATCH 2/8] sql/logictest: clarify retry comment Release note: None --- pkg/sql/logictest/logic.go | 3 +-- pkg/sql/logictest/testdata/logic_test/retry | 2 ++ 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 10b8281e17fa..e68a31e7bd4e 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -313,8 +313,7 @@ import ( // duration. If the test succeeds at any time during that period, it // is considered successful. Otherwise, it is a failure. See // testutils.SucceedsSoon for more information. If run with the -// -rewrite flag, inserts a 2s sleep before executing the query -// once. +// -rewrite flag, the query will be run only once after a 2s sleep. // - async: runs the query asynchronously, marking it as a pending // query using the label parameter as a unique name, to be completed // and validated later with "awaitquery". This is intended for use diff --git a/pkg/sql/logictest/testdata/logic_test/retry b/pkg/sql/logictest/testdata/logic_test/retry index 06d7b7861b2a..5c3909817ceb 100644 --- a/pkg/sql/logictest/testdata/logic_test/retry +++ b/pkg/sql/logictest/testdata/logic_test/retry @@ -4,6 +4,8 @@ SELECT nextval('s') # Unit test for bug #95664, retry on a query producing an error should # try again and succeed if err doesn't happen again. +# NOTE: This test will fail when the --rewrite flag is used because the query +# will only be run once. query I retry SELECT CASE subq.val From 4df9c9b3fa7342dcc6cd518582ee613c2763ac4c Mon Sep 17 00:00:00 2001 From: Herko Lategan Date: Fri, 17 Mar 2023 10:01:39 +0000 Subject: [PATCH 3/8] testutils: move tenant or server logic into test server 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 --- pkg/server/testserver.go | 13 +++++++++++++ pkg/testutils/serverutils/test_server_shim.go | 8 ++++++++ pkg/testutils/testcluster/testcluster.go | 8 ++------ 3 files changed, 23 insertions(+), 6 deletions(-) diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 76aa3b1b222a..a3d0c18eddb1 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -1640,6 +1640,19 @@ func (ts *TestServer) ExecutorConfig() interface{} { return *ts.sqlServer.execCfg } +// StartedDefaultTestTenant is part of the TestServerInterface. +func (ts *TestServer) StartedDefaultTestTenant() bool { + return !ts.cfg.DisableDefaultTestTenant +} + +// TenantOrServer is part of the TestServerInterface. +func (ts *TestServer) TenantOrServer() serverutils.TestTenantInterface { + if ts.StartedDefaultTestTenant() { + return ts.testTenants[0] + } + return ts +} + // TracerI is part of the TestServerInterface. func (ts *TestServer) TracerI() interface{} { return ts.Tracer() diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index 7db5e3a91a9b..d59bdcf2c890 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -275,6 +275,14 @@ type TestServerInterface interface { // TestTenants returns the test tenants associated with the server TestTenants() []TestTenantInterface + + // StartedDefaultTestTenant returns true if the server has started the default + // test tenant. + StartedDefaultTestTenant() bool + + // TenantOrServer returns the default test tenant, if it was started or this + // server if not. + TenantOrServer() TestTenantInterface } // TestServerFactory encompasses the actual implementation of the shim diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index ff38f1ab674d..f5014cdcdf71 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -121,17 +121,13 @@ func (tc *TestCluster) Stopper() *stop.Stopper { // StartedDefaultTestTenant returns whether this cluster started a default // test tenant. func (tc *TestCluster) StartedDefaultTestTenant() bool { - return !tc.Servers[0].Cfg.DisableDefaultTestTenant + return tc.Servers[0].StartedDefaultTestTenant() } // TenantOrServer returns either the ith server in the cluster or the tenant server associated with // the ith server if the cluster started with a default test tenant. func (tc *TestCluster) TenantOrServer(idx int) serverutils.TestTenantInterface { - s := tc.Server(idx) - if tc.StartedDefaultTestTenant() { - return s.TestTenants()[0] - } - return s + return tc.Server(idx).TenantOrServer() } // stopServers stops the stoppers for each individual server in the cluster. From ad436a1293b6372f07008c95a2108c686c001e2e Mon Sep 17 00:00:00 2001 From: adityamaru Date: Sat, 18 Mar 2023 14:20:30 -0400 Subject: [PATCH 4/8] Revert "Revert "backupccl: disallow restore of backups older than the minimum supported binary version"" This reverts commit c510676d474f762051c7f4a9c9899e1e2adfdb29. Release note: None Epic: none --- 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 18e6e45f02bb..07a142b5d5dd 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1427,6 +1427,7 @@ unreserved_keyword ::= | 'UNKNOWN' | 'UNLISTEN' | 'UNLOGGED' + | 'UNSAFE_RESTORE_INCOMPATIBLE_VERSION' | 'UNSET' | 'UNSPLIT' | 'UNTIL' @@ -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 ) )* @@ -3990,6 +3992,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 9e88b2d86e6e..25da98979905 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -8445,20 +8445,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) @@ -8466,45 +8467,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 53292076dc34..af3b58a1b705 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 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 @@ -3769,6 +3769,11 @@ restore_options: { $$.val = &tree.RestoreOptions{VerifyData: true} } +| UNSAFE_RESTORE_INCOMPATIBLE_VERSION + { + $$.val = &tree.RestoreOptions{UnsafeRestoreIncompatibleVersion: true} + } + import_format: name { @@ -16599,6 +16604,7 @@ unreserved_keyword: | UNKNOWN | UNLISTEN | UNLOGGED +| UNSAFE_RESTORE_INCOMPATIBLE_VERSION | UNSET | UNSPLIT | UNTIL @@ -17162,6 +17168,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. From d9b6fe7c875869c76a2eeee6fd1901d5a98d625b Mon Sep 17 00:00:00 2001 From: adityamaru Date: Sat, 18 Mar 2023 18:01:25 -0400 Subject: [PATCH 5/8] roachtest: add `unsafe_restore_incompatible_version` to roachtests 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 Release note: None --- pkg/ccl/workloadccl/fixture.go | 2 +- pkg/cmd/roachtest/tests/schemachange.go | 3 ++- pkg/cmd/roachtest/tests/sqlsmith.go | 10 ++++++++-- pkg/cmd/roachtest/tests/tpc_utils.go | 2 +- pkg/cmd/roachtest/tests/tpcdsvec.go | 5 ++++- 5 files changed, 16 insertions(+), 6 deletions(-) diff --git a/pkg/ccl/workloadccl/fixture.go b/pkg/ccl/workloadccl/fixture.go index e8998722702f..9d88c9e394d5 100644 --- a/pkg/ccl/workloadccl/fixture.go +++ b/pkg/ccl/workloadccl/fixture.go @@ -599,7 +599,7 @@ func RestoreFixture( table := table g.GoCtx(func(ctx context.Context) error { start := timeutil.Now() - restoreStmt := fmt.Sprintf(`RESTORE %s.%s FROM $1 WITH into_db=$2`, genName, table.TableName) + restoreStmt := fmt.Sprintf(`RESTORE %s.%s FROM $1 WITH into_db=$2, unsafe_restore_incompatible_version`, genName, table.TableName) log.Infof(ctx, "Restoring from %s", table.BackupURI) var rows, index, tableBytes int64 var discard interface{} diff --git a/pkg/cmd/roachtest/tests/schemachange.go b/pkg/cmd/roachtest/tests/schemachange.go index 3b2d747e4f92..6e775c6a836f 100644 --- a/pkg/cmd/roachtest/tests/schemachange.go +++ b/pkg/cmd/roachtest/tests/schemachange.go @@ -45,7 +45,8 @@ func registerSchemaChangeDuringKV(r registry.Registry) { m := c.NewMonitor(ctx, c.All()) m.Go(func(ctx context.Context) error { t.Status("loading fixture") - if _, err := db.Exec(`RESTORE DATABASE tpch FROM $1`, fixturePath); err != nil { + if _, err := db.Exec( + `RESTORE DATABASE tpch FROM $1 WITH unsafe_restore_incompatible_version`, fixturePath); err != nil { t.Fatal(err) } return nil diff --git a/pkg/cmd/roachtest/tests/sqlsmith.go b/pkg/cmd/roachtest/tests/sqlsmith.go index 2b5c1fc0eaab..514ea543dbf2 100644 --- a/pkg/cmd/roachtest/tests/sqlsmith.go +++ b/pkg/cmd/roachtest/tests/sqlsmith.go @@ -36,7 +36,10 @@ func registerSQLSmith(r registry.Registry) { "seed": sqlsmith.Setups["seed"], sqlsmith.RandTableSetupName: sqlsmith.Setups[sqlsmith.RandTableSetupName], "tpch-sf1": func(r *rand.Rand) []string { - return []string{`RESTORE TABLE tpch.* FROM 'gs://cockroach-fixtures/workload/tpch/scalefactor=1/backup?AUTH=implicit' WITH into_db = 'defaultdb';`} + return []string{` +RESTORE TABLE tpch.* FROM 'gs://cockroach-fixtures/workload/tpch/scalefactor=1/backup?AUTH=implicit' +WITH into_db = 'defaultdb', unsafe_restore_incompatible_version; +`} }, "tpcc": func(r *rand.Rand) []string { const version = "version=2.1.0,fks=true,interleaved=false,seed=1,warehouses=1" @@ -54,7 +57,10 @@ func registerSQLSmith(r registry.Registry) { } { stmts = append( stmts, - fmt.Sprintf("RESTORE TABLE tpcc.%s FROM 'gs://cockroach-fixtures/workload/tpcc/%[2]s/%[1]s?AUTH=implicit' WITH into_db = 'defaultdb';", + fmt.Sprintf(` +RESTORE TABLE tpcc.%s FROM 'gs://cockroach-fixtures/workload/tpcc/%[2]s/%[1]s?AUTH=implicit' +WITH into_db = 'defaultdb', unsafe_restore_incompatible_version; +`, t, version, ), ) diff --git a/pkg/cmd/roachtest/tests/tpc_utils.go b/pkg/cmd/roachtest/tests/tpc_utils.go index 5e849a406c60..640bd2885a4e 100644 --- a/pkg/cmd/roachtest/tests/tpc_utils.go +++ b/pkg/cmd/roachtest/tests/tpc_utils.go @@ -107,7 +107,7 @@ func loadTPCHDataset( if _, err := db.ExecContext(ctx, `CREATE DATABASE IF NOT EXISTS tpch;`); err != nil { return err } - query := fmt.Sprintf(`RESTORE tpch.* FROM '%s' WITH into_db = 'tpch';`, tpchURL) + query := fmt.Sprintf(`RESTORE tpch.* FROM '%s' WITH into_db = 'tpch', unsafe_restore_incompatible_version;`, tpchURL) _, err = db.ExecContext(ctx, query) return err } diff --git a/pkg/cmd/roachtest/tests/tpcdsvec.go b/pkg/cmd/roachtest/tests/tpcdsvec.go index e6c1f26bc983..471161ddf864 100644 --- a/pkg/cmd/roachtest/tests/tpcdsvec.go +++ b/pkg/cmd/roachtest/tests/tpcdsvec.go @@ -70,7 +70,10 @@ func registerTPCDSVec(r registry.Registry) { } t.Status("restoring TPCDS dataset for Scale Factor 1") if _, err := clusterConn.Exec( - `RESTORE DATABASE tpcds FROM 'gs://cockroach-fixtures/workload/tpcds/scalefactor=1/backup?AUTH=implicit';`, + ` +RESTORE DATABASE tpcds FROM 'gs://cockroach-fixtures/workload/tpcds/scalefactor=1/backup?AUTH=implicit' +WITH unsafe_restore_incompatible_version; +`, ); err != nil { t.Fatal(err) } From 95c90f9731e1dd55d79d04e46aef3dd4ae234731 Mon Sep 17 00:00:00 2001 From: Adam Storm Date: Sun, 19 Mar 2023 18:43:09 -0400 Subject: [PATCH 6/8] multitenant: Increase timeout for kvtenantccl The new TestTenantUpgradeInterlock test takes a long time to run (up to a minute locally) and has started timing (the first instance being in a post-merge arm64 test). 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 Epic: None Fixes: #98986 --- pkg/ccl/kvccl/kvtenantccl/BUILD.bazel | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel b/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel index cbb05d9d763b..ddd28794c9d2 100644 --- a/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel +++ b/pkg/ccl/kvccl/kvtenantccl/BUILD.bazel @@ -45,7 +45,7 @@ go_library( go_test( name = "kvtenantccl_test", - size = "medium", + size = "large", srcs = [ "connector_test.go", "main_test.go", @@ -56,7 +56,7 @@ go_test( "tenant_trace_test.go", "tenant_upgrade_test.go", ], - args = ["-test.timeout=295s"], + args = ["-test.timeout=895s"], embed = [":kvtenantccl"], shard_count = 16, deps = [ From 0b18c6e39859d3f926aa561d20f16c915d546ca3 Mon Sep 17 00:00:00 2001 From: Austen McClernon Date: Thu, 16 Mar 2023 18:00:33 +0000 Subject: [PATCH 7/8] kvserver: remove duplicate logging in s-rebalancer 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 --- pkg/kv/kvserver/store_rebalancer.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index 48153d077f25..621963c944cd 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -592,6 +592,7 @@ func (sr *StoreRebalancer) LogRangeRebalanceOutcome(ctx context.Context, rctx *R "ran out of replicas worth transferring and load %s is still above desired threshold %s; will check again soon", rctx.LocalDesc.Capacity.Load(), rctx.maxThresholds) sr.metrics.ImbalancedStateOverfullOptionsExhausted.Inc(1) + return } // We successfully rebalanced below or equal to the max threshold, @@ -623,9 +624,6 @@ func (sr *StoreRebalancer) RebalanceRanges( ) if candidateReplica == nil { - log.KvDistribution.Infof(ctx, - "ran out of replicas worth transferring and load %s is still above desired threshold %s; will check again soon", - rctx.LocalDesc.Capacity.Load(), rctx.maxThresholds) return NoRebalanceTarget, candidateReplica, voterTargets, nonVoterTargets } From 8e2479543f88fbfa047f9451ea1c6df5158a1f54 Mon Sep 17 00:00:00 2001 From: maryliag Date: Mon, 20 Mar 2023 09:32:05 -0400 Subject: [PATCH 8/8] ui: fix jobs column name 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. --- pkg/ui/workspaces/cluster-ui/src/jobs/jobsPage/jobsTable.tsx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/jobs/jobsPage/jobsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/jobs/jobsPage/jobsTable.tsx index 896a6dd51972..61f305d097ee 100644 --- a/pkg/ui/workspaces/cluster-ui/src/jobs/jobsPage/jobsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/jobs/jobsPage/jobsTable.tsx @@ -193,7 +193,7 @@ export function makeJobsColumns(): ColumnDescriptor[] { style="tableTitle" content={

Date and time the job was last executed.

} > - {jobsColumnLabels.lastModifiedTime} + {jobsColumnLabels.lastExecutionTime} ), cell: job => TimestampToMoment(job?.last_run).format(DATE_FORMAT_24_UTC),