Skip to content

Commit

Permalink
Merge #111387
Browse files Browse the repository at this point in the history
111387: sql: introduce tenant RO setting to allow full access to zone configs r=yuzefovich a=yuzefovich

TLDR: this commit introduces another cluster setting that controls
whether the virtual clusters have unrestricted ability to modify zone
configs.

Previously, we already had `sql.zone_configs.allow_for_secondary_tenant.enabled`
setting (which controls whether VCs can modify zone configs at all), but
even if that was enabled, VCs could not modify constraints and lease
preferences other than locality (and only regions and zones). However,
this restriction can be too cumbersome in UA, so this commit introduces
another cluster setting
`sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled`
that controls whether VCs have full access to zone config modification
(note that region and zone validation is still preserved, and retrieval
of that information is made lazy). The new setting is disabled by
default, but it's overridden in the config profiles `virtual+noapp`,
`virtual+noapp+repl`, `virtual+app+sharedservice`, and
`virtual+app+sharedservice+repl`.

Fixes: #75569.
Fixes: #111358.
Epic: CRDB-26686

Release note: None

Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
craig[bot] and yuzefovich committed Sep 28, 2023
2 parents 2ac6153 + d67afe1 commit e8f27c3
Show file tree
Hide file tree
Showing 13 changed files with 154 additions and 44 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
# LogicTest: 3node-tenant

# 'sql.zone_configs.allow_for_secondary_tenant.enabled' is enabled, but
# 'sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled' is not,
# so we should be able to modify zone configs except when modifying constraints
# other than regions and zones.

statement ok
CREATE TABLE t (k INT PRIMARY KEY);

statement ok
ALTER TABLE t CONFIGURE ZONE USING num_replicas = 5;

# This statement was correctly allowed, but it failed during validation because
# 3node-tenant config doesn't define locality information.
statement error region "us-east1" not found
ALTER TABLE t CONFIGURE ZONE USING constraints = '[+region=us-east1]';

statement error operation is disabled within a virtual cluster
ALTER TABLE t CONFIGURE ZONE USING constraints = '[+ssd]';
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
# LogicTest: 3node-tenant
# tenant-cluster-setting-override-opt: sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled=true

# Both 'sql.zone_configs.allow_for_secondary_tenant.enabled' and
# 'sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled' are
# enabled, so we should be able to modify zone configs in unlimited fashion
# (regions and zones are still subject to validation).

statement ok
CREATE TABLE t (k INT PRIMARY KEY);

statement ok
ALTER TABLE t CONFIGURE ZONE USING num_replicas = 5;

# This statement was correctly allowed, but it failed during validation because
# 3node-tenant config doesn't define locality information.
statement error region "us-east1" not found
ALTER TABLE t CONFIGURE ZONE USING constraints = '[+region=us-east1]';

statement ok
ALTER TABLE t CONFIGURE ZONE USING constraints = '[+ssd]';
14 changes: 14 additions & 0 deletions pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions pkg/configprofiles/profiles.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@ var virtClusterInitTasks = []autoconfigpb.Task{
"SET CLUSTER SETTING trace.redact_at_virtual_cluster_boundary.enabled = false",
// Enable zone config changes in secondary tenants (this ought to be configurable per-tenant, but is not possible yet in v23.1).
"SET CLUSTER SETTING sql.virtual_cluster.feature_access.zone_configs.enabled = true",
"SET CLUSTER SETTING sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled = true",
// Enable multi-region abstractions in secondary tenants.
"SET CLUSTER SETTING sql.virtual_cluster.feature_access.multiregion.enabled = true",
// Disable range coalescing (as long as the problems related
Expand Down
2 changes: 2 additions & 0 deletions pkg/configprofiles/testdata/virtual-app
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ SELECT variable, value FROM [SHOW ALL CLUSTER SETTINGS]
WHERE variable IN (
'trace.redact_at_virtual_cluster_boundary.enabled',
'sql.virtual_cluster.feature_access.zone_configs.enabled',
'sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled',
'sql.virtual_cluster.feature_access.multiregion.enabled',
'spanconfig.range_coalescing.system.enabled',
'spanconfig.range_coalescing.application.enabled',
Expand All @@ -34,6 +35,7 @@ sql.create_virtual_cluster.default_template template
sql.drop_virtual_cluster.enabled false
sql.virtual_cluster.feature_access.multiregion.enabled true
sql.virtual_cluster.feature_access.zone_configs.enabled true
sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled true
trace.redact_at_virtual_cluster_boundary.enabled false

system-sql
Expand Down
2 changes: 2 additions & 0 deletions pkg/configprofiles/testdata/virtual-app-repl
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ SELECT variable, value FROM [SHOW ALL CLUSTER SETTINGS]
WHERE variable IN (
'trace.redact_at_virtual_cluster_boundary.enabled',
'sql.virtual_cluster.feature_access.zone_configs.enabled',
'sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled',
'sql.virtual_cluster.feature_access.multiregion.enabled',
'sql.virtual_cluster.feature_access.multiregion.enabled',
'spanconfig.range_coalescing.system.enabled',
Expand All @@ -35,6 +36,7 @@ sql.create_virtual_cluster.default_template template
sql.drop_virtual_cluster.enabled false
sql.virtual_cluster.feature_access.multiregion.enabled true
sql.virtual_cluster.feature_access.zone_configs.enabled true
sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled true
trace.redact_at_virtual_cluster_boundary.enabled false

system-sql
Expand Down
2 changes: 2 additions & 0 deletions pkg/configprofiles/testdata/virtual-noapp
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ SELECT variable, value FROM [SHOW ALL CLUSTER SETTINGS]
WHERE variable IN (
'trace.redact_at_virtual_cluster_boundary.enabled',
'sql.virtual_cluster.feature_access.zone_configs.enabled',
'sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled',
'sql.virtual_cluster.feature_access.multiregion.enabled',
'sql.virtual_cluster.feature_access.multiregion.enabled',
'spanconfig.range_coalescing.system.enabled',
Expand All @@ -30,6 +31,7 @@ sql.create_virtual_cluster.default_template template
sql.drop_virtual_cluster.enabled false
sql.virtual_cluster.feature_access.multiregion.enabled true
sql.virtual_cluster.feature_access.zone_configs.enabled true
sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled true
trace.redact_at_virtual_cluster_boundary.enabled false

system-sql
Expand Down
2 changes: 2 additions & 0 deletions pkg/configprofiles/testdata/virtual-noapp-repl
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ SELECT variable, value FROM [SHOW ALL CLUSTER SETTINGS]
WHERE variable IN (
'trace.redact_at_virtual_cluster_boundary.enabled',
'sql.virtual_cluster.feature_access.zone_configs.enabled',
'sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled',
'sql.virtual_cluster.feature_access.multiregion.enabled',
'sql.virtual_cluster.feature_access.multiregion.enabled',
'spanconfig.range_coalescing.system.enabled',
Expand All @@ -30,6 +31,7 @@ sql.create_virtual_cluster.default_template template
sql.drop_virtual_cluster.enabled false
sql.virtual_cluster.feature_access.multiregion.enabled true
sql.virtual_cluster.feature_access.zone_configs.enabled true
sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled true
trace.redact_at_virtual_cluster_boundary.enabled false

system-sql
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -3784,13 +3784,13 @@ func (cfg *ExecutorConfig) GetRowMetrics(internal bool) *rowinfra.Metrics {
return cfg.RowMetrics
}

// RequireSystemTenantOrClusterSetting returns a setting disabled error if
// requireSystemTenantOrClusterSetting returns a setting disabled error if
// executed from inside a secondary tenant that does not have the specified
// cluster setting.
func (cfg *ExecutorConfig) RequireSystemTenantOrClusterSetting(
setting *settings.BoolSetting,
func requireSystemTenantOrClusterSetting(
codec keys.SQLCodec, settings *cluster.Settings, setting *settings.BoolSetting,
) error {
if cfg.Codec.ForSystemTenant() || setting.Get(&cfg.Settings.SV) {
if codec.ForSystemTenant() || setting.Get(&settings.SV) {
return nil
}
return errors.WithDetailf(errors.WithHint(
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt_exec_factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -1974,7 +1974,7 @@ func (ef *execFactory) ConstructAlterTableSplit(
return nil, err
}

if err := execCfg.RequireSystemTenantOrClusterSetting(SecondaryTenantSplitAtEnabled); err != nil {
if err := requireSystemTenantOrClusterSetting(execCfg.Codec, execCfg.Settings, SecondaryTenantSplitAtEnabled); err != nil {
return nil, err
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/scatter.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ type scatterNode struct {
// Privileges: INSERT on table.
func (p *planner) Scatter(ctx context.Context, n *tree.Scatter) (planNode, error) {

if err := p.ExecCfg().RequireSystemTenantOrClusterSetting(SecondaryTenantScatterEnabled); err != nil {
if err := requireSystemTenantOrClusterSetting(p.ExecCfg().Codec, p.ExecCfg().Settings, SecondaryTenantScatterEnabled); err != nil {
return nil, err
}

Expand Down
83 changes: 59 additions & 24 deletions pkg/sql/set_zone_config.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
Expand Down Expand Up @@ -260,7 +262,7 @@ func (p *planner) SetZoneConfig(ctx context.Context, n *tree.SetZoneConfig) (pla
return nil, err
}

if err := execCfg.RequireSystemTenantOrClusterSetting(SecondaryTenantZoneConfigsEnabled); err != nil {
if err := requireSystemTenantOrClusterSetting(execCfg.Codec, execCfg.Settings, SecondaryTenantZoneConfigsEnabled); err != nil {
return nil, err
}

Expand Down Expand Up @@ -1015,7 +1017,9 @@ func validateZoneAttrsAndLocalities(
}
return validateZoneAttrsAndLocalitiesForSystemTenant(ctx, ss.ListNodesInternal, zone)
}
return validateZoneLocalitiesForSecondaryTenants(ctx, regionProvider.GetRegions, zone)
return validateZoneLocalitiesForSecondaryTenants(
ctx, regionProvider.GetRegions, zone, execCfg.Codec, execCfg.Settings,
)
}

// validateZoneAttrsAndLocalitiesForSystemTenant performs all the constraint/
Expand Down Expand Up @@ -1070,36 +1074,65 @@ func validateZoneAttrsAndLocalitiesForSystemTenant(
return nil
}

// secondaryTenantsAllZoneConfigsEnabled is an extension of
// SecondaryTenantZoneConfigsEnabled that allows virtual clusters to modify all
// type of constraints in zone configs (i.e. not only zones and regions).
var secondaryTenantsAllZoneConfigsEnabled = settings.RegisterBoolSetting(
settings.TenantReadOnly,
"sql.virtual_cluster.feature_access.zone_configs_unrestricted.enabled",
"enable unrestricted usage of ALTER CONFIGURE ZONE in virtual clusters",
false,
)

// validateZoneLocalitiesForSecondaryTenants performs all the constraint/lease
// preferences validation for secondary tenants. Secondary tenants are only
// allowed to reference locality attributes as they only have access to region
// information via the serverpb.TenantStatusServer. Even then, they're only
// allowed to reference the "region" and "zone" tiers.
// preferences validation for secondary tenants. Unless
// secondaryTenantsAllZoneConfigsEnabled is set to 'true', secondary tenants are
// only allowed to reference locality attributes as they only have access to
// region information via the serverpb.TenantStatusServer. In that case they're
// only allowed to reference the "region" and "zone" tiers.
//
// Unlike the system tenant, we also validate prohibited constraints. This is
// because secondary tenant must operate in the narrow view exposed via the
// serverpb.TenantStatusServer and are not allowed to configure arbitrary
// constraints (required or otherwise).
func validateZoneLocalitiesForSecondaryTenants(
ctx context.Context, getRegions regionsGetter, zone *zonepb.ZoneConfig,
ctx context.Context,
getRegions regionsGetter,
zone *zonepb.ZoneConfig,
codec keys.SQLCodec,
settings *cluster.Settings,
) error {
toValidate := accumulateUniqueConstraints(zone)
resp, err := getRegions(ctx)
if err != nil {
return err
}
regions := make(map[string]struct{})
zones := make(map[string]struct{})
for regionName, regionMeta := range resp.Regions {
regions[regionName] = struct{}{}
for _, zone := range regionMeta.Zones {
zones[zone] = struct{}{}

// rs and zs will be lazily populated with regions and zones, respectively.
// These should not be accessed directly - use getRegionsAndZones helper
// instead.
var rs, zs map[string]struct{}
getRegionsAndZones := func() (regions, zones map[string]struct{}, _ error) {
if rs != nil {
return rs, zs, nil
}
resp, err := getRegions(ctx)
if err != nil {
return nil, nil, err
}
rs, zs = make(map[string]struct{}), make(map[string]struct{})
for regionName, regionMeta := range resp.Regions {
rs[regionName] = struct{}{}
for _, zone := range regionMeta.Zones {
zs[zone] = struct{}{}
}
}
return rs, zs, nil
}

for _, constraint := range toValidate {
switch constraint.Key {
case "zone":
_, zones, err := getRegionsAndZones()
if err != nil {
return err
}
_, found := zones[constraint.Value]
if !found {
return pgerror.Newf(
Expand All @@ -1109,6 +1142,10 @@ func validateZoneLocalitiesForSecondaryTenants(
)
}
case "region":
regions, _, err := getRegionsAndZones()
if err != nil {
return err
}
_, found := regions[constraint.Value]
if !found {
return pgerror.Newf(
Expand All @@ -1118,13 +1155,11 @@ func validateZoneLocalitiesForSecondaryTenants(
)
}
default:
return errors.WithHint(pgerror.Newf(
pgcode.CheckViolation,
"invalid constraint attribute: %q",
constraint.Key,
),
`only "zone" and "region" are allowed`,
)
if err := requireSystemTenantOrClusterSetting(
codec, settings, secondaryTenantsAllZoneConfigsEnabled,
); err != nil {
return err
}
}
}
return nil
Expand Down
39 changes: 25 additions & 14 deletions pkg/sql/set_zone_config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,13 +12,17 @@ package sql

import (
"context"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/config/zonepb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/status/statuspb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/gogo/protobuf/proto"
Expand Down Expand Up @@ -70,6 +74,10 @@ func TestValidateZoneAttrsAndLocalitiesForSecondaryTenants(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()
codec := keys.MakeSQLCodec(serverutils.TestTenantID())
settings := cluster.MakeTestingClusterSettings()

getRegions := func(ctx context.Context) (*serverpb.RegionsResponse, error) {
return &serverpb.RegionsResponse{
Regions: map[string]*serverpb.RegionsResponse_Region{
Expand Down Expand Up @@ -125,33 +133,36 @@ func TestValidateZoneAttrsAndLocalitiesForSecondaryTenants(t *testing.T) {
},
{
cfg: `constraints: ["+rack=us-east1"]`,
errRe: `invalid constraint attribute: "rack"`,
errRe: `operation is disabled within a virtual cluster`,
},
{
cfg: `constraints: ["-rack=us-east1"]`,
errRe: `invalid constraint attribute: "rack"`,
errRe: `operation is disabled within a virtual cluster`,
},
{
cfg: `constraints: ["+ssd"]`,
errRe: `invalid constraint attribute: ""`,
errRe: `operation is disabled within a virtual cluster`,
},
{
cfg: `constraints: ["-ssd"]`,
errRe: `invalid constraint attribute: ""`,
errRe: `operation is disabled within a virtual cluster`,
},
}

for _, tc := range testCases {
var zone zonepb.ZoneConfig
err := yaml.UnmarshalStrict([]byte(tc.cfg), &zone)
require.NoError(t, err)

err = validateZoneLocalitiesForSecondaryTenants(context.Background(), getRegions, &zone)
if tc.errRe == "" {
for _, anyConstraintAllowed := range []bool{false, true} {
secondaryTenantsAllZoneConfigsEnabled.Override(ctx, &settings.SV, anyConstraintAllowed)
for _, tc := range testCases {
var zone zonepb.ZoneConfig
err := yaml.UnmarshalStrict([]byte(tc.cfg), &zone)
require.NoError(t, err)
} else {
require.Error(t, err)
require.True(t, testutils.IsError(err, tc.errRe), "expected %s; got %s", tc.errRe, err.Error())

err = validateZoneLocalitiesForSecondaryTenants(ctx, getRegions, &zone, codec, settings)
if tc.errRe == "" || (anyConstraintAllowed && strings.HasPrefix(tc.errRe, "operation is disabled within a virtual cluster")) {
require.NoError(t, err)
} else {
require.Error(t, err)
require.True(t, testutils.IsError(err, tc.errRe), "expected %s; got %s", tc.errRe, err.Error())
}
}
}
}
Expand Down

0 comments on commit e8f27c3

Please sign in to comment.