From 02d4fbb8c2f44a76ae8a39ce00013cc1bc6017d4 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 26 Sep 2023 19:12:50 -0700 Subject: [PATCH] sql: remove references to closed multi-tenancy issues This commit updates a few places to remove now-closed multi-tenancy issues. Two of those places are gated behind the system tenant, so the issue doesn't matter, and they now use existing issue (even though it doesn't really relate to the features in question). In one place we choose to return "tenant cluster setting" not enabled error to make it more clear (I was just bitten by this because we returned opaque "unimplemented" error). In two other places in comments we update the references to existing issues tracking the corresponding work. Release note: None --- .../zone_config_secondary_tenants_disallowed | 2 +- pkg/cli/interactive_tests/test_demo_partitioning.tcl | 2 +- .../interactive_tests/test_disable_replication.tcl | 4 ++-- pkg/server/serverpb/status.go | 6 ++---- pkg/sql/crdb_internal.go | 7 ++----- pkg/sql/distsql_plan_bulk.go | 2 +- pkg/sql/exec_util.go | 5 ++++- pkg/sql/logictest/logictestbase/logictestbase.go | 2 +- pkg/sql/multitenant_admin_function_test.go | 8 ++++---- pkg/sql/set_zone_config.go | 11 ++--------- pkg/workload/workloadsql/workloadsql.go | 2 +- 11 files changed, 21 insertions(+), 30 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/zone_config_secondary_tenants_disallowed b/pkg/ccl/logictestccl/testdata/logic_test/zone_config_secondary_tenants_disallowed index a0c80080f165..ff14d1b05307 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/zone_config_secondary_tenants_disallowed +++ b/pkg/ccl/logictestccl/testdata/logic_test/zone_config_secondary_tenants_disallowed @@ -1,7 +1,7 @@ # LogicTest: 3node-tenant # tenant-cluster-setting-override-opt: sql.virtual_cluster.feature_access.zone_configs.enabled=false -statement error pq: unimplemented: operation is unsupported within a virtual cluster +statement error pq: operation is disabled within a virtual cluster\nHINT: Feature was disabled by the system operator.\nDETAIL: Feature flag: sql.virtual_cluster.feature_access.zone_configs.enabled ALTER TABLE t CONFIGURE ZONE USING num_replicas = 5; statement error setting sql.virtual_cluster.feature_access.zone_configs.enabled is only settable by the operator diff --git a/pkg/cli/interactive_tests/test_demo_partitioning.tcl b/pkg/cli/interactive_tests/test_demo_partitioning.tcl index 3d330f51ae4f..08ff1cc05ac7 100644 --- a/pkg/cli/interactive_tests/test_demo_partitioning.tcl +++ b/pkg/cli/interactive_tests/test_demo_partitioning.tcl @@ -132,7 +132,7 @@ end_test start_test "Expect an error if geo-partitioning is requested with multitenant mode" send "$argv demo --no-line-editor --geo-partitioned-replicas --log-dir=logs \r" # expect a failure -eexpect "operation is unsupported within a virtual cluster" +eexpect "operation is disabled within a virtual cluster" eexpect $prompt end_test diff --git a/pkg/cli/interactive_tests/test_disable_replication.tcl b/pkg/cli/interactive_tests/test_disable_replication.tcl index 00c5d33cd5db..1a51a4671ec8 100644 --- a/pkg/cli/interactive_tests/test_disable_replication.tcl +++ b/pkg/cli/interactive_tests/test_disable_replication.tcl @@ -7,8 +7,8 @@ send "PS1=':''/# '\r" eexpect ":/# " start_test "Check that demo disables replication properly" -# Disable multitenant until zone configs are properly enabled in tenants. -# See 67679 for more details. +# Disable multitenant until default zone config is properly propagated to +# virtual clusters. See #110003 for more details. send "$argv demo --multitenant=false -e 'show zone configuration for range default'\r" eexpect "num_replicas = 1" eexpect ":/# " diff --git a/pkg/server/serverpb/status.go b/pkg/server/serverpb/status.go index 029c9647b694..707dba28de1f 100644 --- a/pkg/server/serverpb/status.go +++ b/pkg/server/serverpb/status.go @@ -92,10 +92,8 @@ type TenantStatusServer interface { // OptionalNodesStatusServer returns the wrapped NodesStatusServer, if it is // available. If it is not, an error referring to the optionally supplied issues // is returned. -func (s *OptionalNodesStatusServer) OptionalNodesStatusServer( - issue int, -) (NodesStatusServer, error) { - v, err := s.w.OptionalErr(issue) +func (s *OptionalNodesStatusServer) OptionalNodesStatusServer() (NodesStatusServer, error) { + v, err := s.w.OptionalErr(errorutil.FeatureNotAvailableToNonSystemTenantsIssue) if err != nil { return nil, err } diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 121874b457a7..fbd4e201cb46 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -91,7 +91,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/vtable" "github.com/cockroachdb/cockroach/pkg/util/admission/admissionpb" "github.com/cockroachdb/cockroach/pkg/util/duration" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/json" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -5426,8 +5425,7 @@ CREATE TABLE crdb_internal.kv_node_status ( if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil { return err } - ss, err := p.extendedEvalCtx.NodesStatusServer.OptionalNodesStatusServer( - errorutil.FeatureNotAvailableToNonSystemTenantsIssue) + ss, err := p.extendedEvalCtx.NodesStatusServer.OptionalNodesStatusServer() if err != nil { return err } @@ -5540,8 +5538,7 @@ CREATE TABLE crdb_internal.kv_store_status ( if err := p.CheckPrivilege(ctx, syntheticprivilege.GlobalPrivilegeObject, privilege.VIEWCLUSTERMETADATA); err != nil { return err } - ss, err := p.ExecCfg().NodesStatusServer.OptionalNodesStatusServer( - errorutil.FeatureNotAvailableToNonSystemTenantsIssue) + ss, err := p.ExecCfg().NodesStatusServer.OptionalNodesStatusServer() if err != nil { return err } diff --git a/pkg/sql/distsql_plan_bulk.go b/pkg/sql/distsql_plan_bulk.go index 29cfdee96fba..3432eb09bbf3 100644 --- a/pkg/sql/distsql_plan_bulk.go +++ b/pkg/sql/distsql_plan_bulk.go @@ -64,7 +64,7 @@ func (dsp *DistSQLPlanner) setupAllNodesPlanningSystem( planCtx := dsp.NewPlanningCtxWithOracle(ctx, evalCtx, nil /* planner */, nil, /* txn */ DistributionTypeAlways, oracle, localityFilter) - ss, err := execCfg.NodesStatusServer.OptionalNodesStatusServer(47900) + ss, err := execCfg.NodesStatusServer.OptionalNodesStatusServer() if err != nil { return planCtx, []base.SQLInstanceID{dsp.gatewaySQLInstanceID}, nil //nolint:returnerrcheck } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index fa977a58f164..5d232d6c18da 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -3793,5 +3793,8 @@ func (cfg *ExecutorConfig) RequireSystemTenantOrClusterSetting( if cfg.Codec.ForSystemTenant() || setting.Get(&cfg.Settings.SV) { return nil } - return errors.Newf("tenant cluster setting %s disabled", setting.Name()) + return errors.WithDetailf(errors.WithHint( + errors.New("operation is disabled within a virtual cluster"), + "Feature was disabled by the system operator."), + "Feature flag: %s", setting.Name()) } diff --git a/pkg/sql/logictest/logictestbase/logictestbase.go b/pkg/sql/logictest/logictestbase/logictestbase.go index 6d139e83da11..577e7fbca805 100644 --- a/pkg/sql/logictest/logictestbase/logictestbase.go +++ b/pkg/sql/logictest/logictestbase/logictestbase.go @@ -324,7 +324,7 @@ var LogicTestConfigs = []TestClusterConfig{ // Have to disable the default test tenant here as there are test run in // this mode which try to modify zone configurations and we're more // restrictive in the way we allow zone configs to be modified by - // secondary tenants. See #75569 for more info. + // secondary tenants. See #100787 for more info. // // TODO(#76378): We should review this choice. Zone configs have // been supported for secondary tenants since v22.2. diff --git a/pkg/sql/multitenant_admin_function_test.go b/pkg/sql/multitenant_admin_function_test.go index 919417520903..6fe6c0df64da 100644 --- a/pkg/sql/multitenant_admin_function_test.go +++ b/pkg/sql/multitenant_admin_function_test.go @@ -470,7 +470,7 @@ func TestMultiTenantAdminFunction(t *testing.T) { result: [][]string{{ignore, "/1", maxTimestamp}}, }, secondaryWithoutClusterSetting: tenantExpected{ - errorMessage: "tenant cluster setting sql.virtual_cluster.feature_access.manual_range_split.enabled disabled", + errorMessage: "operation is disabled within a virtual cluster", }, queryClusterSetting: sql.SecondaryTenantSplitAtEnabled, setupCapability: bcap(tenantcapabilities.CanAdminSplit, false), @@ -484,7 +484,7 @@ func TestMultiTenantAdminFunction(t *testing.T) { result: [][]string{{"\xf0\x8a\x89", "/1", maxTimestamp}}, }, secondaryWithoutClusterSetting: tenantExpected{ - errorMessage: "tenant cluster setting sql.virtual_cluster.feature_access.manual_range_split.enabled disabled", + errorMessage: "operation is disabled within a virtual cluster", }, queryClusterSetting: sql.SecondaryTenantSplitAtEnabled, setupCapability: bcap(tenantcapabilities.CanAdminSplit, false), @@ -571,7 +571,7 @@ func TestMultiTenantAdminFunction(t *testing.T) { result: [][]string{{ignore, ignore}}, }, secondaryWithoutClusterSetting: tenantExpected{ - errorMessage: "tenant cluster setting sql.virtual_cluster.feature_access.manual_range_scatter.enabled disabled", + errorMessage: "operation is disabled within a virtual cluster", }, secondaryWithoutCapability: tenantExpected{ errorMessage: `does not have capability "can_admin_scatter"`, @@ -588,7 +588,7 @@ func TestMultiTenantAdminFunction(t *testing.T) { result: [][]string{{"\xf0\x8a", "/Table/104/2"}}, }, secondaryWithoutClusterSetting: tenantExpected{ - errorMessage: "tenant cluster setting sql.virtual_cluster.feature_access.manual_range_scatter.enabled disabled", + errorMessage: "operation is disabled within a virtual cluster", }, queryClusterSetting: sql.SecondaryTenantScatterEnabled, setupCapability: bcap(tenantcapabilities.CanAdminScatter, false), diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go index 97c16ec013cf..3c33c89802a4 100644 --- a/pkg/sql/set_zone_config.go +++ b/pkg/sql/set_zone_config.go @@ -35,7 +35,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/log/logpb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -262,10 +261,7 @@ func (p *planner) SetZoneConfig(ctx context.Context, n *tree.SetZoneConfig) (pla } if err := execCfg.RequireSystemTenantOrClusterSetting(SecondaryTenantZoneConfigsEnabled); err != nil { - // Return an unimplemented error here instead of referencing the cluster - // setting here as zone configurations for secondary tenants are intended to - // be hidden. - return nil, errorutil.UnsupportedUnderClusterVirtualization(MultitenancyZoneCfgIssueNo) + return nil, err } if err := checkPrivilegeForSetZoneConfig(ctx, p, n.ZoneSpecifier); err != nil { @@ -1013,7 +1009,7 @@ func validateZoneAttrsAndLocalities( return nil } if execCfg.Codec.ForSystemTenant() { - ss, err := execCfg.NodesStatusServer.OptionalNodesStatusServer(MultitenancyZoneCfgIssueNo) + ss, err := execCfg.NodesStatusServer.OptionalNodesStatusServer() if err != nil { return err } @@ -1134,9 +1130,6 @@ func validateZoneLocalitiesForSecondaryTenants( return nil } -// MultitenancyZoneCfgIssueNo points to the multitenancy zone config issue number. -const MultitenancyZoneCfgIssueNo = 49854 - type zoneConfigUpdate struct { id descpb.ID zoneConfig catalog.ZoneConfig diff --git a/pkg/workload/workloadsql/workloadsql.go b/pkg/workload/workloadsql/workloadsql.go index 5e9ec23bc532..fe1674052770 100644 --- a/pkg/workload/workloadsql/workloadsql.go +++ b/pkg/workload/workloadsql/workloadsql.go @@ -122,7 +122,7 @@ func Split(ctx context.Context, db *gosql.DB, table workload.Table, concurrency // Test that we can actually perform a scatter. if _, err := db.Exec("ALTER TABLE system.jobs SCATTER"); err != nil { - if strings.Contains(err.Error(), "tenant cluster setting sql.virtual_cluster.feature_access.manual_range_scatter.enabled disabled") { + if strings.Contains(err.Error(), "operation is disabled within a virtual cluster") { log.Infof(ctx, `skipping workload splits; can't scatter on tenants'`) //nolint:returnerrcheck return nil