Skip to content

Commit

Permalink
upgrademanager: defer retrieval of cluster ID
Browse files Browse the repository at this point in the history
Previously, the upgrade manager's constructor took the cluster ID
directly.

However, the cluster ID may not be initialised at the point of
constructing the manager. This isn't typically a problem because in
most cases, the only upgrade that _require_ the cluster ID doesn't
actually take the cluster ID from the manager. Rather, it gets the
cluster ID from the execution context of the migration job on
resumption.

But, tests such as TestRetriesWithExponentialBackoff set the
DontUseJobs testing hook. In this case, the migration manager runs the
upgrades directly, passing its own copy of the cluster ID, which is
unitialized.

To fix this, we pass the cluster ID container rather than the cluster
ID so that we can defer retrieving it until we actually run the
migrations, at which point it should be set.

Informs #112763

Release note: none
  • Loading branch information
stevendanna committed Oct 31, 2023
1 parent 06add59 commit 0e48843
Show file tree
Hide file tree
Showing 3 changed files with 7 additions and 9 deletions.
2 changes: 1 addition & 1 deletion pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1264,7 +1264,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
knobs, _ := cfg.TestingKnobs.UpgradeManager.(*upgradebase.TestingKnobs)
upgradeMgr = upgrademanager.NewManager(
systemDeps, leaseMgr, cfg.circularInternalExecutor, jobRegistry, codec,
cfg.Settings, clusterIDForSQL.Get(), knobs,
cfg.Settings, clusterIDForSQL, knobs,
)
execCfg.UpgradeJobDeps = upgradeMgr
execCfg.VersionUpgradeHook = upgradeMgr.Migrate
Expand Down
1 change: 0 additions & 1 deletion pkg/upgrade/upgrademanager/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ go_library(
"//pkg/util/log",
"//pkg/util/startup",
"//pkg/util/stop",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_logtags//:logtags",
"@com_github_cockroachdb_redact//:redact",
Expand Down
13 changes: 6 additions & 7 deletions pkg/upgrade/upgrademanager/manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/startup"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/cockroachdb/redact"
Expand All @@ -59,7 +58,7 @@ type Manager struct {
codec keys.SQLCodec
settings *cluster.Settings
knobs upgradebase.TestingKnobs
clusterID uuid.UUID
clusterID *base.ClusterIDContainer
}

// GetUpgrade returns the upgrade associated with this key.
Expand Down Expand Up @@ -89,7 +88,7 @@ func NewManager(
jr *jobs.Registry,
codec keys.SQLCodec,
settings *cluster.Settings,
clusterID uuid.UUID,
clusterID *base.ClusterIDContainer,
testingKnobs *upgradebase.TestingKnobs,
) *Manager {
var knobs upgradebase.TestingKnobs
Expand Down Expand Up @@ -213,7 +212,7 @@ func (m *Manager) RunPermanentUpgrades(ctx context.Context, upToVersion roachpb.
// because upgrades run in order.
latest := permanentUpgrades[len(permanentUpgrades)-1]
lastVer := latest.Version()
enterpriseEnabled := base.CCLDistributionAndEnterpriseEnabled(m.settings, m.clusterID)
enterpriseEnabled := base.CCLDistributionAndEnterpriseEnabled(m.settings, m.clusterID.Get())
lastUpgradeCompleted, err := startup.RunIdempotentWithRetryEx(ctx,
m.deps.Stopper.ShouldQuiesce(),
"check if migration completed",
Expand Down Expand Up @@ -708,7 +707,7 @@ func (m *Manager) runMigration(
InternalExecutor: m.ie,
JobRegistry: m.jr,
TestingKnobs: &m.knobs,
ClusterID: m.clusterID,
ClusterID: m.clusterID.Get(),
}); err != nil {
return err
}
Expand Down Expand Up @@ -761,7 +760,7 @@ func (m *Manager) getOrCreateMigrationJob(
) (alreadyCompleted, alreadyExisting bool, jobID jobspb.JobID, _ error) {
newJobID := m.jr.MakeJobID()
if err := m.deps.DB.Txn(ctx, func(ctx context.Context, txn isql.Txn) (err error) {
enterpriseEnabled := base.CCLDistributionAndEnterpriseEnabled(m.settings, m.clusterID)
enterpriseEnabled := base.CCLDistributionAndEnterpriseEnabled(m.settings, m.clusterID.Get())
alreadyCompleted, err = migrationstable.CheckIfMigrationCompleted(
ctx, version, txn.KV(), txn, enterpriseEnabled, migrationstable.ConsistentRead,
)
Expand Down Expand Up @@ -896,7 +895,7 @@ func (m *Manager) checkPreconditions(ctx context.Context, versions []roachpb.Ver
LeaseManager: m.lm,
InternalExecutor: m.ie,
JobRegistry: m.jr,
ClusterID: m.clusterID,
ClusterID: m.clusterID.Get(),
}); err != nil {
return errors.Wrapf(
err,
Expand Down

0 comments on commit 0e48843

Please sign in to comment.