Skip to content

Commit

Permalink
upgrade: automatically bump the system DB version
Browse files Browse the repository at this point in the history
Rather than making authors of an upgrade remember that they should bump
the version of the system DB schema, now it will be bumped automatically
after any upgrade step that has a migration or if upgrading to the final
cluster version for a release. A test is added to make sure the system
database version after upgrading is equal to the bootstrap version.

This change results in more work, since it means that the version will
be bumped when it is not strictly necessary to do so. But since it
makes it harder to forget a manual step, it seems worth doing.

Release note: None
  • Loading branch information
rafiss committed Apr 29, 2024
1 parent 4e178de commit aa47896
Show file tree
Hide file tree
Showing 10 changed files with 89 additions and 51 deletions.
30 changes: 10 additions & 20 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -197,28 +197,13 @@ const (
// options lagging_ranges_threshold and lagging_ranges_polling_interval.
TODODelete_V23_2_ChangefeedLaggingRangesOpts

// ***************************************************************************
// WHERE TO ADD VERSION GATES DURING 23.2 STABILITY?
// ---------------------------------------------------------------------------
// If version gate is for 23.2 (to be backported to release-23.2):
// Then add new gate above this comment (immediately above this comment).
// If version gate is for 24.1 (upcoming 24.1 development):
// Then add new gate at the end (immediately above the "Add new versions
// here" comment).
// ***************************************************************************

// V23_2 is CockroachDB v23.2. It's used for all v23.2.x patch releases.
V23_2

// V24_1Start demarcates the start of cluster versions stepped through during
// the process of upgrading from 23.2 to 24.1.
V24_1Start

// *************************************************
// Step (1) Add new versions here.
// Do not add new versions to a patch release.
// *************************************************

// V24_1_DropPayloadAndProgressFromSystemJobsTable drop the unused payload and
// progress columns from system.jobs table.
V24_1_DropPayloadAndProgressFromSystemJobsTable
Expand Down Expand Up @@ -268,6 +253,11 @@ const (
// the system tenant to ensure it is a superset of secondary tenants.
V24_1_AddSpanCounts

// *************************************************
// Step (1) Add new versions above this comment.
// Do not add new versions to a patch release.
// *************************************************

numKeys
)

Expand Down Expand Up @@ -308,11 +298,6 @@ var versionTable = [numKeys]roachpb.Version{
// v24.1 versions. Internal versions must be even.
V24_1Start: {Major: 23, Minor: 2, Internal: 2},

// *************************************************
// Step (2): Add new versions here.
// Do not add new versions to a patch release.
// *************************************************

V24_1_DropPayloadAndProgressFromSystemJobsTable: {Major: 23, Minor: 2, Internal: 4},

V24_1_MigrateOldStylePTSRecords: {Major: 23, Minor: 2, Internal: 6},
Expand All @@ -327,6 +312,11 @@ var versionTable = [numKeys]roachpb.Version{
V24_1_EstimatedMVCCStatsInSplit: {Major: 23, Minor: 2, Internal: 22},
V24_1_ReplicatedLockPipelining: {Major: 23, Minor: 2, Internal: 24},
V24_1_AddSpanCounts: {Major: 23, Minor: 2, Internal: 26},

// *************************************************
// Step (2): Add new versions above this comment.
// Do not add new versions to a patch release.
// *************************************************
}

// Latest is always the highest version key. This is the maximum logical cluster
Expand Down
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -510,6 +510,7 @@ go_test(
"//pkg/spanconfig",
"//pkg/sql",
"//pkg/sql/appstatspb",
"//pkg/sql/catalog/descs",
"//pkg/sql/execinfrapb",
"//pkg/sql/isql",
"//pkg/sql/roleoption",
Expand Down
40 changes: 39 additions & 1 deletion pkg/server/migration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,13 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand All @@ -32,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -318,7 +321,9 @@ func TestMigrationPurgeOutdatedReplicas(t *testing.T) {

// TestUpgradeHappensAfterMigration is a regression test to ensure that
// upgrades run prior to attempting to upgrade the cluster to the current
// version.
// version. It will also verify that any migrations that modify the system
// database schema properly update the SystemDatabaseSchemaVersion on the
// system database descriptor.
func TestUpgradeHappensAfterMigrations(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down Expand Up @@ -349,6 +354,25 @@ func TestUpgradeHappensAfterMigrations(t *testing.T) {
},
},
})

internalDB := s.ApplicationLayer().InternalDB().(descs.DB)
err := internalDB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
systemDBDesc, err := txn.Descriptors().ByID(txn.KV()).Get().Database(ctx, keys.SystemDatabaseID)
if err != nil {
return err
}
systemDBVersion := systemDBDesc.DatabaseDesc().GetSystemDatabaseSchemaVersion()
if clusterversion.MinSupported.Version().Less(*systemDBVersion) {
// NB: When MinSupported is changed to 24_2, this check can change to
// an equality check. This is because 24.2 is the first release where
// https://github.com/cockroachdb/cockroach/issues/121914 has been
// resolved.
return errors.Newf("before upgrade, expected system database version (%v) to be less than or equal to clusterversion.MinSupported (%v)", *systemDBVersion, clusterversion.MinSupported.Version())
}
return nil
})
require.NoError(t, err)

close(automaticUpgrade)
sr := sqlutils.MakeSQLRunner(db)

Expand All @@ -359,5 +383,19 @@ func TestUpgradeHappensAfterMigrations(t *testing.T) {
SELECT version = crdb_internal.node_executable_version()
FROM [SHOW CLUSTER SETTING version]`,
[][]string{{"true"}})

err = internalDB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
systemDBDesc, err := txn.Descriptors().ByID(txn.KV()).Get().Database(ctx, keys.SystemDatabaseID)
if err != nil {
return err
}
systemDBVersion := systemDBDesc.DatabaseDesc().GetSystemDatabaseSchemaVersion()
if !systemDBVersion.Equal(clusterversion.Latest.Version()) {
return errors.Newf("after upgrade, expected system database version (%v) to be equal to clusterversion.Latest (%v)", *systemDBVersion, clusterversion.Latest.Version())
}
return nil
})
require.NoError(t, err)

s.Stopper().Stop(context.Background())
}
1 change: 1 addition & 0 deletions pkg/upgrade/upgrademanager/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_library(
"//pkg/server/settingswatcher",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/catalog/descs",
"//pkg/sql/catalog/lease",
"//pkg/sql/isql",
"//pkg/sql/protoreflect",
Expand Down
34 changes: 34 additions & 0 deletions pkg/upgrade/upgrademanager/manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/settingswatcher"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/lease"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/protoreflect"
Expand Down Expand Up @@ -522,6 +523,14 @@ func (m *Manager) Migrate(
}
}

// Bump the version of the system database schema if there was a migration
// or if this is the final version for a release.
if exists || (clusterVersion.Equal(clusterversion.Latest.Version()) && clusterVersion.IsFinal()) {
if err := bumpSystemDatabaseSchemaVersion(ctx, cv, m.deps.DB); err != nil {
return err
}
}

if m.knobs.InterlockPausePoint == upgradebase.AfterMigration {
m.postToPauseChannelAndWaitForResume(ctx)
}
Expand Down Expand Up @@ -549,6 +558,31 @@ func (m *Manager) Migrate(
return nil
}

// bumpSystemDatabaseSchemaVersion bumps the SystemDatabaseSchemaVersion
// field for the system database descriptor. It is called after every upgrade
// step.
func bumpSystemDatabaseSchemaVersion(
ctx context.Context, cs clusterversion.ClusterVersion, descDB descs.DB,
) error {
return descDB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
systemDBDesc, err := txn.Descriptors().MutableByID(txn.KV()).Database(ctx, keys.SystemDatabaseID)
if err != nil {
return err
}
if sv := systemDBDesc.GetSystemDatabaseSchemaVersion(); sv != nil {
if cs.Version.Less(*sv) {
return errors.AssertionFailedf(
"new system schema version (%#v) is lower than previous system schema version (%#v)",
cs.Version,
*sv,
)
}
}
systemDBDesc.SystemDatabaseSchemaVersion = &cs.Version
return txn.Descriptors().WriteDesc(ctx, false /* kvTrace */, systemDBDesc, txn.KV())
})
}

// bumpClusterVersion will invoke the BumpClusterVersion rpc on every node
// until the cluster is stable.
func bumpClusterVersion(
Expand Down
26 changes: 0 additions & 26 deletions pkg/upgrade/upgrades/schema_changes.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/upgrade"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -412,28 +411,3 @@ func onlyHasColumnFamily(
}
return true, nil
}

// bumpSystemDatabaseSchemaVersion bumps the SystemDatabaseSchemaVersion
// field for the system database descriptor. It should be called at the end
// of any upgrade that creates or modifies the schema of any system table.
func bumpSystemDatabaseSchemaVersion(
ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps,
) error {
return d.DB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
systemDBDesc, err := txn.Descriptors().MutableByName(txn.KV()).Database(ctx, catconstants.SystemDatabaseName)
if err != nil {
return err
}
if sv := systemDBDesc.GetSystemDatabaseSchemaVersion(); sv != nil {
if cs.Version.Less(*sv) {
return errors.AssertionFailedf(
"new system schema version (%#v) is lower than previous system schema version (%#v)",
cs.Version,
*sv,
)
}
}
systemDBDesc.SystemDatabaseSchemaVersion = &cs.Version
return txn.Descriptors().WriteDesc(ctx, false /* kvTrace */, systemDBDesc, txn.KV())
})
}
2 changes: 1 addition & 1 deletion pkg/upgrade/upgrades/v24_1_add_span_counts.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,5 +28,5 @@ func addSpanCountTable(
if err := createSystemTable(ctx, d.DB, d.Settings, d.Codec, systemschema.SpanCountTable, tree.LocalityLevelTable); err != nil {
return err
}
return bumpSystemDatabaseSchemaVersion(ctx, cv, d)
return nil
}
Original file line number Diff line number Diff line change
Expand Up @@ -45,5 +45,5 @@ func hidePayloadProgressFromSystemJobs(
return err
}

return bumpSystemDatabaseSchemaVersion(ctx, cv, deps)
return nil
}
2 changes: 1 addition & 1 deletion pkg/upgrade/upgrades/v24_1_session_based_lease.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,5 +149,5 @@ func upgradeSystemLeasesDescriptor(
}); err != nil {
return err
}
return bumpSystemDatabaseSchemaVersion(ctx, version, deps)
return nil
}
2 changes: 1 addition & 1 deletion pkg/upgrade/upgrades/v24_1_system_database.go
Original file line number Diff line number Diff line change
Expand Up @@ -129,5 +129,5 @@ ALTER DATABASE system SURVIVE REGION FAILURE;
}
}

return bumpSystemDatabaseSchemaVersion(ctx, cv, deps)
return nil
}

0 comments on commit aa47896

Please sign in to comment.