Skip to content

Commit

Permalink
pkg/server: support tenant auto-upgrade
Browse files Browse the repository at this point in the history
Previously, tenant upgrades in UA required a user to issue a `SET CLUSTER SETTING version =`
statement to finalize an upgrade. This UX is different from what we have in single-tenant
SH/Dedicated deployments in that we have auto upgrade in the later that starts an attempt
to finalize cluster version after every node startup incase the node was started with a new
binary version that all nodes now support upgrading to.

In UA, we have two differences:

1. What to upgrade?
    - In a multi-tenant deployment, the storage and sql layers are upgraded separately.
    - The storage layer upgrade finalization is still handled by the existing auto upgrade logic.
    - In this change, we ensure that the sql layer is also auto-upgraded when possible.
2. When to upgrade?
    - In a single-tenant deployment, all layers share the same binary version and cluster version.
      Hence, an upgrade attempt is only needed when a new node starts to ensure that the cluster is
      auto-upgraded if the new binary version supports an upgrade.
    - In a multi-tenant deployment, in addition to the condition above, the sql server upgrade is
      also constrained by the storage cluster version. It is possible for all SQL instances to have
      binary versions that support an upgrade but the upgrade will still be blocked by the storage
      cluster version if it’s equal to the current tenant cluster version.

This code change does the following:

1. Adds logic to run a SQL server upgrade attempt (mostly adopted from the original auto upgrade code)
   within the following ordered constraints (previously we merged #98830 to make getting the binary
   versions of instances easier):
    - Ensure that upgrade is not blocked by `preserve_downgrade_option`.
    - Exit if tenant cluster version is equal to storage cluster version [upgrade already completed].
    - Upgrade to storage cluster version if the binary version of all SQL instances supports that.
    - Exit if tenant cluster version is equal to the minimum instance binary version [upgrade blocked
      due to too low binary version].
    - Upgrade to the minimum instance binary version.

2. Runs the logic above when a SQL server is started.
    - This covers the case where a SQL server binary upgrade allows for an upgrade to the
      tenant cluster version.

3. Checks for change in storage cluster version every 30 seconds and starts an upgrade attempt if
   it was changed.
    - This covers the case where the binary versions of all SQL instances allow for an upgrade
      but it’s blocked due to the storage cluster version.

Release note: None
Epic: CRDB-20860
  • Loading branch information
healthy-pod committed May 1, 2023
1 parent ce40ce7 commit be11008
Show file tree
Hide file tree
Showing 8 changed files with 352 additions and 3 deletions.
3 changes: 3 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/upgradeccl/BUILD.bazel
Expand Up @@ -8,6 +8,7 @@ go_test(
"tenant_upgrade_test.go",
],
args = ["-test.timeout=295s"],
shard_count = 3,
tags = ["ccl_test"],
deps = [
"//pkg/base",
Expand All @@ -24,10 +25,12 @@ go_test(
"//pkg/sql/sqlinstance/instancestorage",
"//pkg/sql/sqlliveness/slinstance",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/upgrade",
"//pkg/upgrade/upgradebase",
"//pkg/util",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/randutil",
Expand Down
122 changes: 119 additions & 3 deletions pkg/ccl/kvccl/kvtenantccl/upgradeccl/tenant_upgrade_test.go
Expand Up @@ -25,16 +25,120 @@ import (
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/sql/sqlinstance/instancestorage"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slinstance"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/upgrade"
"github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/stretchr/testify/require"
)

func TestTenantAutoUpgrade(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
settings := cluster.MakeTestingClusterSettingsWithVersions(
clusterversion.TestingBinaryVersion,
clusterversion.TestingBinaryMinSupportedVersion,
false, // initializeVersion
)
// Initialize the version to the BinaryMinSupportedVersion.
require.NoError(t, clusterversion.Initialize(ctx,
clusterversion.TestingBinaryMinSupportedVersion, &settings.SV))
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
// Test validates tenant behavior. No need for the default test
// tenant.
DefaultTestTenant: base.TestTenantDisabled,
Settings: settings,
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DisableAutomaticVersionUpgrade: make(chan struct{}),
BinaryVersionOverride: clusterversion.TestingBinaryMinSupportedVersion,
},
},
},
})
defer tc.Stopper().Stop(ctx)

expectedInitialTenantVersion := clusterversion.TestingBinaryMinSupportedVersion
expectedFinalTenantVersion := clusterversion.TestingBinaryVersion
connectToTenant := func(t *testing.T, addr string) (_ *gosql.DB, cleanup func()) {
pgURL, cleanupPGUrl := sqlutils.PGUrl(t, addr, "Tenant", url.User(username.RootUser))
tenantDB, err := gosql.Open("postgres", pgURL.String())
require.NoError(t, err)
return tenantDB, func() {
tenantDB.Close()
cleanupPGUrl()
}
}
mkTenant := func(t *testing.T, id uint64) (tenantDB *gosql.DB, cleanup func()) {
settings := cluster.MakeTestingClusterSettingsWithVersions(
clusterversion.TestingBinaryVersion,
clusterversion.TestingBinaryMinSupportedVersion,
false, // initializeVersion
)
// Initialize the version to the minimum it could be.
require.NoError(t, clusterversion.Initialize(ctx,
expectedInitialTenantVersion, &settings.SV))
tenantArgs := base.TestTenantArgs{
TenantID: roachpb.MustMakeTenantID(id),
TestingKnobs: base.TestingKnobs{},
Settings: settings,
}
tenant, err := tc.Server(0).StartTenant(ctx, tenantArgs)
require.NoError(t, err)
tenantDB, cleanup = connectToTenant(t, tenant.SQLAddr())
return tenantDB, cleanup
}

// Create a tenant before upgrading anything and verify its version.
const initialTenantID = 10
tenantDB, cleanup := mkTenant(t, initialTenantID)
tenantRunner := sqlutils.MakeSQLRunner(tenantDB)

// Ensure that the tenant works.
tenantRunner.CheckQueryResults(t, "SHOW CLUSTER SETTING version",
[][]string{{expectedInitialTenantVersion.String()}})
tenantRunner.Exec(t, "CREATE TABLE t (i INT PRIMARY KEY)")
tenantRunner.Exec(t, "INSERT INTO t VALUES (1), (2)")

// Upgrade the host cluster.
sqlutils.MakeSQLRunner(tc.ServerConn(0)).Exec(t,
"SET CLUSTER SETTING version = $1",
clusterversion.TestingBinaryVersion.String())

// Ensure that the tenant still works.
tenantRunner.CheckQueryResults(t, "SELECT * FROM t", [][]string{{"1"}, {"2"}})
// Wait for the tenant to auto-finalize.
tenantRunner.SucceedsSoonDuration = time.Second * 90
if skip.Stress() || util.RaceEnabled {
tenantRunner.SucceedsSoonDuration = time.Second * 180
}
tenantRunner.CheckQueryResultsRetry(t, "SHOW CLUSTER SETTING version",
[][]string{{expectedFinalTenantVersion.String()}})

// Restart the tenant and ensure that the version is correct.
cleanup()
{
tenantServer, err := tc.Server(0).StartTenant(ctx, base.TestTenantArgs{
TenantID: roachpb.MustMakeTenantID(initialTenantID),
})
require.NoError(t, err)
tenantDB, cleanup = connectToTenant(t, tenantServer.SQLAddr())
defer cleanup()
tenantRunner = sqlutils.MakeSQLRunner(tenantDB)
}
tenantRunner.CheckQueryResults(t, "SELECT * FROM t", [][]string{{"1"}, {"2"}})
tenantRunner.CheckQueryResults(t, "SHOW CLUSTER SETTING version",
[][]string{{expectedFinalTenantVersion.String()}})

}

// TestTenantUpgrade exercises the case where a system tenant is in a
// non-finalized version state and creates a tenant. The test ensures
// that the newly created tenant begins in that same version.
Expand Down Expand Up @@ -95,9 +199,13 @@ func TestTenantUpgrade(t *testing.T) {
require.NoError(t, clusterversion.Initialize(ctx,
expectedInitialTenantVersion, &settings.SV))
tenantArgs := base.TestTenantArgs{
TenantID: roachpb.MustMakeTenantID(id),
TestingKnobs: base.TestingKnobs{},
Settings: settings,
TenantID: roachpb.MustMakeTenantID(id),
TestingKnobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DisableAutomaticVersionUpgrade: make(chan struct{}),
},
},
Settings: settings,
}
tenant, err := tc.Server(0).StartTenant(ctx, tenantArgs)
require.NoError(t, err)
Expand Down Expand Up @@ -139,6 +247,11 @@ func TestTenantUpgrade(t *testing.T) {
{
tenantServer, err := tc.Server(0).StartTenant(ctx, base.TestTenantArgs{
TenantID: roachpb.MustMakeTenantID(initialTenantID),
TestingKnobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DisableAutomaticVersionUpgrade: make(chan struct{}),
},
},
})
require.NoError(t, err)
initialTenant, cleanup = connectToTenant(t, tenantServer.SQLAddr())
Expand Down Expand Up @@ -275,6 +388,9 @@ func TestTenantUpgradeFailure(t *testing.T) {
SpanConfig: &spanconfig.TestingKnobs{
ManagerDisableJobCreation: true,
},
Server: &server.TestingKnobs{
DisableAutomaticVersionUpgrade: make(chan struct{}),
},
UpgradeManager: &upgradebase.TestingKnobs{
DontUseJobs: true,
RegistryOverride: func(v roachpb.Version) (upgradebase.Upgrade, bool) {
Expand Down
Expand Up @@ -291,6 +291,9 @@ func TestTenantUpgradeInterlock(t *testing.T) {
tenantArgs := base.TestTenantArgs{
TenantID: id,
TestingKnobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DisableAutomaticVersionUpgrade: make(chan struct{}),
},
JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(),
UpgradeManager: &upgradebase.TestingKnobs{
InterlockPausePoint: test.pausePoint,
Expand Down Expand Up @@ -424,6 +427,11 @@ func TestTenantUpgradeInterlock(t *testing.T) {
Stopper: otherServerStopper,
TenantID: tenantID,
Settings: otherServerSettings,
TestingKnobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DisableAutomaticVersionUpgrade: make(chan struct{}),
},
},
})

var otherTenantRunner *sqlutils.SQLRunner
Expand Down
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Expand Up @@ -75,6 +75,7 @@ go_library(
"stop_trigger.go",
"tcp_keepalive_manager.go",
"tenant.go",
"tenant_auto_upgrade.go",
"tenant_migration.go",
"testing_knobs.go",
"testserver.go",
Expand Down
1 change: 1 addition & 0 deletions pkg/server/auto_upgrade.go
Expand Up @@ -116,6 +116,7 @@ const (
upgradeDisabledByConfiguration
upgradeBlockedDueToError
upgradeBlockedDueToMixedVersions
upgradeBlockedDueToLowBinaryVersion
)

// upgradeStatus lets the main checking loop know if we should do upgrade,
Expand Down
8 changes: 8 additions & 0 deletions pkg/server/server_sql.go
Expand Up @@ -1698,6 +1698,14 @@ func (s *SQLServer) preStart(
}
}))

if err := s.startAttemptUpgrade(ctx); err != nil {
return errors.Wrap(err, "cannot start tenant auto upgrade task")
}

if err := s.startAutoUpgradeOnStorageUpgrade(ctx); err != nil {
return errors.Wrap(err, "cannot start tenant auto upgrade checker task")
}

return nil
}

Expand Down

0 comments on commit be11008

Please sign in to comment.