Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
91768: utilccl: don't require the org name in checkLicense r=andreimatei a=andreimatei

The license checking functions required the org name to be passed in, even though it was also taking the settings and the org name comes from a setting. This patch removes the need to pass in that arg. This aims to make it a bit easier for random call sites to check for a license, which currently is a bit tedious.

The change is in license_check.go. Everything else is removing the args from cascading callers.

Release note: None
Epic: None

Co-authored-by: Andrei Matei <[email protected]>
  • Loading branch information
craig[bot] and andreimatei committed Nov 14, 2022
2 parents a00fbe9 + 1b0f6ec commit 3f00311
Show file tree
Hide file tree
Showing 28 changed files with 40 additions and 61 deletions.
2 changes: 1 addition & 1 deletion pkg/base/license.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ var errEnterpriseNotEnabled = errors.New("OSS binaries do not include enterprise
// enable it.
//
// This function is overridden by an init hook in CCL builds.
var CheckEnterpriseEnabled = func(_ *cluster.Settings, _ uuid.UUID, org, feature string) error {
var CheckEnterpriseEnabled = func(_ *cluster.Settings, _ uuid.UUID, feature string) error {
return errEnterpriseNotEnabled // nb: this is squarely in the hot path on OSS builds
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/alter_backup_schedule.go
Original file line number Diff line number Diff line change
Expand Up @@ -627,7 +627,7 @@ func makeAlterBackupScheduleSpec(
}

enterpriseCheckErr := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(), p.ExecCfg().Organization(),
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(),
"BACKUP INTO LATEST")
spec.isEnterpriseUser = enterpriseCheckErr == nil

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -561,7 +561,7 @@ func (b *backupResumer) Resume(ctx context.Context, execCtx interface{}) error {

// Collect telemetry, once per backup after resolving its destination.
lic := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(), p.ExecCfg().Organization(), "",
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(), "",
) != nil
collectTelemetry(ctx, m, initialDetails, details, lic, b.job.ID())
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -485,7 +485,7 @@ func checkPrivilegesForBackup(

func requireEnterprise(execCfg *sql.ExecutorConfig, feature string) error {
if err := utilccl.CheckEnterpriseEnabled(
execCfg.Settings, execCfg.NodeInfo.LogicalClusterID(), execCfg.Organization(),
execCfg.Settings, execCfg.NodeInfo.LogicalClusterID(),
fmt.Sprintf("BACKUP with %s", feature),
); err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -769,7 +769,7 @@ func makeScheduledBackupSpec(
}

enterpriseCheckErr := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(), p.ExecCfg().Organization(),
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(),
"BACKUP INTO LATEST")
spec.isEnterpriseUser = enterpriseCheckErr == nil

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -2074,7 +2074,7 @@ func planDatabaseModifiersForRestore(
return nil, nil, nil
}
if err := multiregionccl.CheckClusterSupportsMultiRegion(
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(), p.ExecCfg().Organization(),
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(),
); err != nil {
return nil, nil, errors.WithHintf(
err,
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/changefeed_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -553,7 +553,7 @@ func createChangefeedJobRecord(

if scope, ok := opts.GetMetricScope(); ok {
if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(), p.ExecCfg().Organization(), "CHANGEFEED",
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(), "CHANGEFEED",
); err != nil {
return nil, errors.Wrapf(err,
"use of %q option requires enterprise license.", changefeedbase.OptMetricsScope)
Expand Down Expand Up @@ -586,7 +586,7 @@ func createChangefeedJobRecord(
}

if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(), p.ExecCfg().Organization(), "CHANGEFEED",
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(), "CHANGEFEED",
); err != nil {
return nil, err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/gssapiccl/gssapi.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ func authGSS(
// their GSS configuration is correct. That is, the presence of this error
// message means they have a correctly functioning GSS/Kerberos setup,
// but now need to enable enterprise features.
return utilccl.CheckEnterpriseEnabled(execCfg.Settings, execCfg.NodeInfo.LogicalClusterID(), execCfg.Organization(), "GSS authentication")
return utilccl.CheckEnterpriseEnabled(execCfg.Settings, execCfg.NodeInfo.LogicalClusterID(), "GSS authentication")
})
return behaviors, nil
}
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/jwtauthccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ go_library(
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/pgwire",
"//pkg/sql/pgwire/identmap",
"//pkg/util/log",
Expand Down
4 changes: 1 addition & 3 deletions pkg/ccl/jwtauthccl/authentication_jwt.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/identmap"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -186,8 +185,7 @@ func (authenticator *jwtAuthenticator) ValidateJWTLogin(
"token issued with an audience of %s", parsedToken.Audience())
}

org := sql.ClusterOrganization.Get(&st.SV)
if err = utilccl.CheckEnterpriseEnabled(st, authenticator.clusterUUID, org, "JWT authentication"); err != nil {
if err = utilccl.CheckEnterpriseEnabled(st, authenticator.clusterUUID, "JWT authentication"); err != nil {
return err
}

Expand Down
5 changes: 1 addition & 4 deletions pkg/ccl/kvccl/kvfollowerreadsccl/boundedstaleness.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/asof"
Expand All @@ -24,11 +23,9 @@ import (
)

func checkBoundedStalenessEnabled(evalCtx *eval.Context) error {
st := evalCtx.Settings
return utilccl.CheckEnterpriseEnabled(
st,
evalCtx.Settings,
evalCtx.ClusterID,
sql.ClusterOrganization.Get(&st.SV),
"bounded staleness",
)
}
Expand Down
6 changes: 2 additions & 4 deletions pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,15 +74,13 @@ func getGlobalReadsLead(clock *hlc.Clock) time.Duration {
// reads is enabled, returning a detailed error if not. It is not suitable for
// use in hot paths since a new error may be instantiated on each call.
func checkEnterpriseEnabled(logicalClusterID uuid.UUID, st *cluster.Settings) error {
org := sql.ClusterOrganization.Get(&st.SV)
return utilccl.CheckEnterpriseEnabled(st, logicalClusterID, org, "follower reads")
return utilccl.CheckEnterpriseEnabled(st, logicalClusterID, "follower reads")
}

// isEnterpriseEnabled is faster than checkEnterpriseEnabled, and suitable
// for hot paths.
func isEnterpriseEnabled(logicalClusterID uuid.UUID, st *cluster.Settings) bool {
org := sql.ClusterOrganization.Get(&st.SV)
return utilccl.IsEnterpriseEnabled(st, logicalClusterID, org, "follower reads")
return utilccl.IsEnterpriseEnabled(st, logicalClusterID, "follower reads")
}

func checkFollowerReadsEnabled(logicalClusterID uuid.UUID, st *cluster.Settings) bool {
Expand Down
9 changes: 2 additions & 7 deletions pkg/ccl/multiregionccl/multiregion.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@ func initializeMultiRegionMetadata(
descIDGenerator eval.DescIDGenerator,
settings *cluster.Settings,
clusterID uuid.UUID,
clusterOrganization string,
liveRegions sql.LiveClusterRegions,
goal tree.SurvivalGoal,
primaryRegion catpb.RegionName,
Expand All @@ -45,7 +44,7 @@ func initializeMultiRegionMetadata(
secondaryRegion catpb.RegionName,
) (*multiregion.RegionConfig, error) {
if err := CheckClusterSupportsMultiRegion(
settings, clusterID, clusterOrganization,
settings, clusterID,
); err != nil {
return nil, err
}
Expand Down Expand Up @@ -130,13 +129,10 @@ func initializeMultiRegionMetadata(

// CheckClusterSupportsMultiRegion returns whether the current cluster supports
// multi-region features.
func CheckClusterSupportsMultiRegion(
settings *cluster.Settings, clusterID uuid.UUID, organization string,
) error {
func CheckClusterSupportsMultiRegion(settings *cluster.Settings, clusterID uuid.UUID) error {
return utilccl.CheckEnterpriseEnabled(
settings,
clusterID,
organization,
"multi-region features",
)
}
Expand All @@ -147,7 +143,6 @@ func getMultiRegionEnumAddValuePlacement(
if err := utilccl.CheckEnterpriseEnabled(
execCfg.Settings,
execCfg.NodeInfo.LogicalClusterID(),
execCfg.Organization(),
"ADD REGION",
); err != nil {
return tree.AlterTypeAddValue{}, err
Expand Down
1 change: 0 additions & 1 deletion pkg/ccl/oidcccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ go_library(
"//pkg/server/telemetry",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/ui",
"//pkg/util/log",
"//pkg/util/protoutil",
Expand Down
4 changes: 1 addition & 3 deletions pkg/ccl/oidcccl/authentication_oidc.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/ui"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
Expand Down Expand Up @@ -365,8 +364,7 @@ var ConfigureOIDC = func(
return
}

org := sql.ClusterOrganization.Get(&st.SV)
if err := utilccl.CheckEnterpriseEnabled(st, cluster, org, "OIDC"); err != nil {
if err := utilccl.CheckEnterpriseEnabled(st, cluster, "OIDC"); err != nil {
http.Error(w, err.Error(), http.StatusBadRequest)
return
}
Expand Down
3 changes: 1 addition & 2 deletions pkg/ccl/partitionccl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -346,8 +346,7 @@ func createPartitioning(
allowedNewColumnNames []tree.Name,
allowImplicitPartitioning bool,
) (newImplicitCols []catalog.Column, newPartitioning catpb.PartitioningDescriptor, err error) {
org := sql.ClusterOrganization.Get(&st.SV)
if err := utilccl.CheckEnterpriseEnabled(st, evalCtx.ClusterID, org, "partitions"); err != nil {
if err := utilccl.CheckEnterpriseEnabled(st, evalCtx.ClusterID, "partitions"); err != nil {
return nil, newPartitioning, err
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/streamingccl/streamingest/stream_ingest_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func newStreamIngestManagerWithPrivilegesCheck(

execCfg := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig)
enterpriseCheckErr := utilccl.CheckEnterpriseEnabled(
execCfg.Settings, execCfg.NodeInfo.LogicalClusterID(), execCfg.Organization(), "REPLICATION")
execCfg.Settings, execCfg.NodeInfo.LogicalClusterID(), "REPLICATION")
if enterpriseCheckErr != nil {
return nil, pgerror.Wrap(enterpriseCheckErr,
pgcode.InsufficientPrivilege, "replication requires enterprise license")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ func ingestionPlanHook(
defer span.Finish()

if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(), p.ExecCfg().Organization(),
p.ExecCfg().Settings, p.ExecCfg().NodeInfo.LogicalClusterID(),
"RESTORE FROM REPLICATION STREAM",
); err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/streamingccl/streamproducer/replication_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ func newReplicationStreamManagerWithPrivilegesCheck(
execCfg := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig)

enterpriseCheckErr := utilccl.CheckEnterpriseEnabled(
execCfg.Settings, execCfg.NodeInfo.LogicalClusterID(), execCfg.Organization(), "REPLICATION")
execCfg.Settings, execCfg.NodeInfo.LogicalClusterID(), "REPLICATION")
if enterpriseCheckErr != nil {
return nil, pgerror.Wrap(enterpriseCheckErr,
pgcode.InsufficientPrivilege, "replication requires enterprise license")
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/utilccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ go_library(
"//pkg/server",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/sem/builtins/builtinsregistry",
Expand Down
12 changes: 7 additions & 5 deletions pkg/ccl/utilccl/license_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
Expand Down Expand Up @@ -114,8 +115,8 @@ func ApplyTenantLicense() error {
// The ClusterID argument should be the tenant-specific logical
// cluster ID. is not used for the check itself; it is merely embedded
// in the URL displayed in the error message.
func CheckEnterpriseEnabled(st *cluster.Settings, cluster uuid.UUID, org, feature string) error {
return checkEnterpriseEnabledAt(st, timeutil.Now(), cluster, org, feature, true /* withDetails */)
func CheckEnterpriseEnabled(st *cluster.Settings, cluster uuid.UUID, feature string) error {
return checkEnterpriseEnabledAt(st, timeutil.Now(), cluster, feature, true /* withDetails */)
}

// IsEnterpriseEnabled returns whether the requested enterprise feature is
Expand All @@ -126,9 +127,9 @@ func CheckEnterpriseEnabled(st *cluster.Settings, cluster uuid.UUID, org, featur
// The ClusterID argument should be the tenant-specific logical
// cluster ID. is not used for the check itself; it is merely embedded
// in the URL displayed in the error message.
func IsEnterpriseEnabled(st *cluster.Settings, cluster uuid.UUID, org, feature string) bool {
func IsEnterpriseEnabled(st *cluster.Settings, cluster uuid.UUID, feature string) bool {
return checkEnterpriseEnabledAt(
st, timeutil.Now(), cluster, org, feature, false /* withDetails */) == nil
st, timeutil.Now(), cluster, feature, false /* withDetails */) == nil
}

func init() {
Expand Down Expand Up @@ -184,7 +185,7 @@ func updateMetricWithLicenseTTL(
}

func checkEnterpriseEnabledAt(
st *cluster.Settings, at time.Time, cluster uuid.UUID, org, feature string, withDetails bool,
st *cluster.Settings, at time.Time, cluster uuid.UUID, feature string, withDetails bool,
) error {
if atomic.LoadInt32(&enterpriseStatus) == enterpriseEnabled {
return nil
Expand All @@ -193,6 +194,7 @@ func checkEnterpriseEnabledAt(
if err != nil {
return err
}
org := sql.ClusterOrganization.Get(&st.SV)
return check(license, at, cluster, org, feature, withDetails)
}

Expand Down
18 changes: 9 additions & 9 deletions pkg/ccl/utilccl/license_check_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ func TestSettingAndCheckingLicense(t *testing.T) {
if err := setLicense(ctx, updater, tc.lic); err != nil {
t.Fatal(err)
}
err := checkEnterpriseEnabledAt(st, tc.checkTime, tc.checkCluster, "", "", true)
err := checkEnterpriseEnabledAt(st, tc.checkTime, tc.checkCluster, "", true)
if !testutils.IsError(err, tc.err) {
l, _ := decode(tc.lic)
t.Fatalf("%d: lic %v, update by %T, checked by %s at %s, got %q", i, l, updater, tc.checkCluster, tc.checkTime, err)
Expand Down Expand Up @@ -204,11 +204,11 @@ func TestApplyTenantLicenseWithLicense(t *testing.T) {

settings := cluster.MakeClusterSettings()

require.Error(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.False(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.Error(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), ""))
require.False(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), ""))
require.NoError(t, ApplyTenantLicense())
require.NoError(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.True(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.NoError(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), ""))
require.True(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), ""))
}

func TestApplyTenantLicenseWithoutLicense(t *testing.T) {
Expand All @@ -219,11 +219,11 @@ func TestApplyTenantLicenseWithoutLicense(t *testing.T) {
envutil.ClearEnvCache()
require.False(t, ok)

require.Error(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.False(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.Error(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), ""))
require.False(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), ""))
require.NoError(t, ApplyTenantLicense())
require.Error(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.False(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), "", ""))
require.Error(t, CheckEnterpriseEnabled(settings, uuid.MakeV4(), ""))
require.False(t, IsEnterpriseEnabled(settings, uuid.MakeV4(), ""))
}

func TestApplyTenantLicenseWithInvalidLicense(t *testing.T) {
Expand Down
2 changes: 0 additions & 2 deletions pkg/server/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -1978,11 +1978,9 @@ func (s *adminServer) Cluster(
// Check if enterprise features are enabled. We currently test for the
// feature "BACKUP", although enterprise licenses do not yet distinguish
// between different features.
organization := sql.ClusterOrganization.Get(&s.server.st.SV)
enterpriseEnabled := base.CheckEnterpriseEnabled(
s.server.st,
s.server.rpcContext.LogicalClusterID.Get(),
organization,
"BACKUP") == nil

return &serverpb.ClusterResponse{
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1435,7 +1435,7 @@ func TestClusterAPI(t *testing.T) {
// Override server license check.
if enterpriseOn {
old := base.CheckEnterpriseEnabled
base.CheckEnterpriseEnabled = func(_ *cluster.Settings, _ uuid.UUID, _, _ string) error {
base.CheckEnterpriseEnabled = func(_ *cluster.Settings, _ uuid.UUID, _ string) error {
return nil
}
defer func() { base.CheckEnterpriseEnabled = old }()
Expand Down
3 changes: 1 addition & 2 deletions pkg/server/testserver.go
Original file line number Diff line number Diff line change
Expand Up @@ -514,9 +514,8 @@ func (ts *TestServer) TestTenants() []serverutils.TestTenantInterface {
// enterprise enabled build. This is due to licensing restrictions on the MT
// capabilities.
func (ts *TestServer) maybeStartDefaultTestTenant(ctx context.Context) error {
org := sql.ClusterOrganization.Get(&ts.st.SV)
clusterID := ts.sqlServer.execCfg.NodeInfo.LogicalClusterID
if err := base.CheckEnterpriseEnabled(ts.st, clusterID(), org, "SQL servers"); err != nil {
if err := base.CheckEnterpriseEnabled(ts.st, clusterID(), "SQL servers"); err != nil {
// If not enterprise enabled, we won't be able to use SQL Servers so eat
// the error and return without creating/starting a SQL server.
ts.cfg.DisableDefaultTestTenant = true
Expand Down
2 changes: 0 additions & 2 deletions pkg/sql/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -353,7 +353,6 @@ var InitializeMultiRegionMetadataCCL = func(
descIDGenerator eval.DescIDGenerator,
settings *cluster.Settings,
clusterID uuid.UUID,
clusterOrganization string,
liveClusterRegions LiveClusterRegions,
survivalGoal tree.SurvivalGoal,
primaryRegion catpb.RegionName,
Expand Down Expand Up @@ -447,7 +446,6 @@ func (p *planner) maybeInitializeMultiRegionMetadata(
p.EvalContext().DescIDGenerator,
p.EvalContext().Settings,
p.ExecCfg().NodeInfo.LogicalClusterID(),
p.ExecCfg().Organization(),
liveRegions,
survivalGoal,
catpb.RegionName(primaryRegion),
Expand Down
Loading

0 comments on commit 3f00311

Please sign in to comment.