Skip to content

Commit

Permalink
sql: remove limitations on tenants setting cluster settings
Browse files Browse the repository at this point in the history
This change also removes the limits in the SET CLUSTER SETTING
code path that prevented tenants writing to their settings table
before those changes would have had any effect.

Release note: none.
  • Loading branch information
dt authored and ajwerner committed Dec 30, 2020
1 parent 8e27190 commit 20855d0
Show file tree
Hide file tree
Showing 4 changed files with 12 additions and 48 deletions.
2 changes: 0 additions & 2 deletions pkg/ccl/serverccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ go_test(
"//pkg/server",
"//pkg/server/serverpb",
"//pkg/sql",
"//pkg/sql/pgwire/pgcode",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
Expand All @@ -36,7 +35,6 @@ go_test(
"//pkg/util/log",
"//pkg/util/randutil",
"//pkg/util/timeutil",
"@com_github_lib_pq//:pq",
"@com_github_stretchr_testify//require",
"@org_golang_x_crypto//bcrypt",
],
Expand Down
7 changes: 4 additions & 3 deletions pkg/ccl/serverccl/server_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,21 +10,18 @@ package serverccl

import (
"context"
"errors"
"io/ioutil"
"net/http"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/httputil"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/lib/pq"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -75,10 +72,14 @@ func TestTenantCannotSetClusterSetting(t *testing.T) {
db := serverutils.StartTenant(t, tc.Server(0), base.TestTenantArgs{TenantID: roachpb.MakeTenantID(10), AllowSettingClusterSettings: false})
defer db.Close()
_, err := db.Exec(`SET CLUSTER SETTING sql.defaults.vectorize=off`)
require.NoError(t, err)
/* TODO(dt): re-introduce when system-settings are prevented in tenants.
_, err = db.Exec(`SET CLUSTER SETTING kv.snapshot_rebalance.max_rate = '2MiB';`)
var pqErr *pq.Error
ok := errors.As(err, &pqErr)
require.True(t, ok, "expected err to be a *pq.Error but is of type %T. error is: %v", err)
require.Equal(t, pq.ErrorCode(pgcode.InsufficientPrivilege.String()), pqErr.Code, "err %v has unexpected code", err)
*/
}

func TestTenantUnauthenticatedAccess(t *testing.T) {
Expand Down
6 changes: 0 additions & 6 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -906,12 +906,6 @@ var _ base.ModuleTestingKnobs = &TenantTestingKnobs{}
// ModuleTestingKnobs implements the base.ModuleTestingKnobs interface.
func (*TenantTestingKnobs) ModuleTestingKnobs() {}

// CanSetClusterSettings is a helper method that returns whether the tenant can
// set in-memory cluster settings.
func (k *TenantTestingKnobs) CanSetClusterSettings() bool {
return k != nil && k.ClusterSettingsUpdater != nil
}

// BackupRestoreTestingKnobs contains knobs for backup and restore behavior.
type BackupRestoreTestingKnobs struct {
// AllowImplicitAccess allows implicit access to data sources for non-admin
Expand Down
45 changes: 8 additions & 37 deletions pkg/sql/set_cluster_setting.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,12 +77,6 @@ func checkPrivilegesForSetting(ctx context.Context, p *planner, name string, act
func (p *planner) SetClusterSetting(
ctx context.Context, n *tree.SetClusterSetting,
) (planNode, error) {
if !p.execCfg.TenantTestingKnobs.CanSetClusterSettings() && !p.execCfg.Codec.ForSystemTenant() {
// Setting cluster settings is disabled for phase 2 tenants if a test does
// not explicitly allow for setting in-memory cluster settings.
return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "only the system tenant can SET CLUSTER SETTING")
}

name := strings.ToLower(n.Name)
st := p.EvalContext().Settings
v, ok := settings.Lookup(name, settings.LookupForLocalAccess)
Expand All @@ -99,12 +93,6 @@ func (p *planner) SetClusterSetting(
return nil, errors.AssertionFailedf("expected writable setting, got %T", v)
}

if _, ok := setting.(*settings.VersionSetting); ok && p.execCfg.TenantTestingKnobs.CanSetClusterSettings() {
// A tenant that is allowed to set in-memory cluster settings is
// attempting to set the cluster version setting, which is disallowed.
return nil, pgerror.Newf(pgcode.InsufficientPrivilege, "only the system tenant can set version settings")
}

var value tree.TypedExpr
if n.Value != nil {
// For DEFAULT, let the value reference be nil. That's a RESET in disguise.
Expand Down Expand Up @@ -174,31 +162,6 @@ func (n *setClusterSettingNode) startExec(params runParams) error {
if !params.p.ExtendedEvalContext().TxnImplicit {
return errors.Errorf("SET CLUSTER SETTING cannot be used inside a transaction")
}

if !params.p.execCfg.Codec.ForSystemTenant() {
// Sanity check that this tenant is able to set in-memory settings.
if !params.p.execCfg.TenantTestingKnobs.CanSetClusterSettings() {
return errors.Errorf("tenants cannot set cluster settings, this permission should have been checked at plan time")
}
var encodedValue string
if n.value == nil {
encodedValue = n.setting.EncodedDefault()
} else {
value, err := n.value.Eval(params.p.EvalContext())
if err != nil {
return err
}
if _, ok := n.setting.(*settings.VersionSetting); ok {
return errors.Errorf("tenants cannot change cluster version setting, this should've been checked at plan time")
}
encodedValue, err = toSettingString(params.ctx, n.st, n.name, n.setting, value, nil /* prev */)
if err != nil {
return err
}
}
return params.p.execCfg.TenantTestingKnobs.ClusterSettingsUpdater.Set(n.name, encodedValue, n.setting.Typ())
}

execCfg := params.extendedEvalCtx.ExecCfg
var expectedEncodedValue string
if err := execCfg.DB.Txn(params.ctx, func(ctx context.Context, txn *kv.Txn) error {
Expand Down Expand Up @@ -272,6 +235,14 @@ func (n *setClusterSettingNode) startExec(params runParams) error {
); err != nil {
return err
}

if params.p.execCfg.TenantTestingKnobs != nil {
if err := params.p.execCfg.TenantTestingKnobs.ClusterSettingsUpdater.Set(
n.name, encoded, n.setting.Typ(),
); err != nil {
return err
}
}
}

// Report tracked cluster settings via telemetry.
Expand Down

0 comments on commit 20855d0

Please sign in to comment.