Skip to content

Commit

Permalink
settings: add SystemOnly flag
Browse files Browse the repository at this point in the history
This change adds a flag to indicate a setting is only applicable to the
system tenant, such as those that control storage layer behaviors, and
thus should not be set by guest tenants (where it would be ignored).

Release note: none.
  • Loading branch information
dt authored and ajwerner committed Dec 30, 2020
1 parent 20855d0 commit a4fbace
Show file tree
Hide file tree
Showing 12 changed files with 76 additions and 4 deletions.
2 changes: 2 additions & 0 deletions pkg/ccl/serverccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ go_test(
"//pkg/server",
"//pkg/server/serverpb",
"//pkg/sql",
"//pkg/sql/pgwire/pgcode",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
Expand All @@ -35,6 +36,7 @@ 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
5 changes: 3 additions & 2 deletions pkg/ccl/serverccl/server_sql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,18 +10,21 @@ 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 @@ -73,13 +76,11 @@ func TestTenantCannotSetClusterSetting(t *testing.T) {
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
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/store_snapshot.go
Original file line number Diff line number Diff line change
Expand Up @@ -845,7 +845,7 @@ var rebalanceSnapshotRate = settings.RegisterByteSizeSetting(
"the rate limit (bytes/sec) to use for rebalance and upreplication snapshots",
envutil.EnvOrDefaultBytes("COCKROACH_PREEMPTIVE_SNAPSHOT_RATE", 8<<20),
validatePositive,
).WithPublic()
).WithPublic().WithSystemOnly()

// recoverySnapshotRate is the rate at which Raft-initiated spanshots can be
// sent. Ideally, one would never see a Raft-initiated snapshot; we'd like all
Expand All @@ -858,7 +858,7 @@ var recoverySnapshotRate = settings.RegisterByteSizeSetting(
"the rate limit (bytes/sec) to use for recovery snapshots",
envutil.EnvOrDefaultBytes("COCKROACH_RAFT_SNAPSHOT_RATE", 8<<20),
validatePositive,
).WithPublic()
).WithPublic().WithSystemOnly()

// snapshotSenderBatchSize is the size that key-value batches are allowed to
// grow to during Range snapshots before being sent to the receiver. This limit
Expand Down
9 changes: 9 additions & 0 deletions pkg/settings/bool.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,15 @@ func (b *BoolSetting) WithPublic() *BoolSetting {
return b
}

// WithSystemOnly marks this setting as system-only and can be chained.
func (b *BoolSetting) WithSystemOnly() *BoolSetting {
b.common.systemOnly = true
return b
}

// Defeat the linter.
var _ = (*BoolSetting).WithSystemOnly

// RegisterBoolSetting defines a new setting with type bool.
func RegisterBoolSetting(key, desc string, defaultValue bool) *BoolSetting {
setting := &BoolSetting{defaultValue: defaultValue}
Expand Down
6 changes: 6 additions & 0 deletions pkg/settings/byte_size.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,12 @@ func (b *ByteSizeSetting) WithPublic() *ByteSizeSetting {
return b
}

// WithSystemOnly marks this setting as system-only and can be chained.
func (b *ByteSizeSetting) WithSystemOnly() *ByteSizeSetting {
b.common.systemOnly = true
return b
}

// RegisterByteSizeSetting defines a new setting with type bytesize and any
// supplied validation function(s).
func RegisterByteSizeSetting(
Expand Down
9 changes: 9 additions & 0 deletions pkg/settings/duration.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,15 @@ func (d *DurationSetting) WithPublic() *DurationSetting {
return d
}

// WithSystemOnly marks this setting as system-only and can be chained.
func (d *DurationSetting) WithSystemOnly() *DurationSetting {
d.common.systemOnly = true
return d
}

// Defeat the linter.
var _ = (*DurationSetting).WithSystemOnly

// RegisterDurationSetting defines a new setting with type duration.
func RegisterDurationSetting(
key, desc string, defaultValue time.Duration, validateFns ...func(time.Duration) error,
Expand Down
9 changes: 9 additions & 0 deletions pkg/settings/enum.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,15 @@ func (e *EnumSetting) WithPublic() *EnumSetting {
return e
}

// WithSystemOnly indicates system-usage only and can be chained.
func (e *EnumSetting) WithSystemOnly() *EnumSetting {
e.common.systemOnly = true
return e
}

// Defeat the linter.
var _ = (*EnumSetting).WithSystemOnly

// RegisterEnumSetting defines a new setting with type int.
func RegisterEnumSetting(
key, desc string, defaultValue string, enumValues map[int64]string,
Expand Down
9 changes: 9 additions & 0 deletions pkg/settings/float.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,15 @@ func (f *FloatSetting) setToDefault(sv *Values) {
}
}

// WithSystemOnly indicates system-usage only and can be chained.
func (f *FloatSetting) WithSystemOnly() *FloatSetting {
f.common.systemOnly = true
return f
}

// Defeat the linter.
var _ = (*FloatSetting).WithSystemOnly

// RegisterFloatSetting defines a new setting with type float.
func RegisterFloatSetting(
key, desc string, defaultValue float64, validateFns ...func(float64) error,
Expand Down
9 changes: 9 additions & 0 deletions pkg/settings/int.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,15 @@ func (i *IntSetting) WithPublic() *IntSetting {
return i
}

// WithSystemOnly system-only usage and can be chained.
func (i *IntSetting) WithSystemOnly() *IntSetting {
i.common.systemOnly = true
return i
}

// Defeat the linter.
var _ = (*IntSetting).WithSystemOnly

// PositiveInt can be passed to RegisterIntSetting
func PositiveInt(v int64) error {
if v < 1 {
Expand Down
5 changes: 5 additions & 0 deletions pkg/settings/masked.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,3 +47,8 @@ func (s *MaskedSetting) Description() string {
func (s *MaskedSetting) Typ() string {
return s.setting.Typ()
}

// SystemOnly returns the underlying setting's SystemOnly.
func (s *MaskedSetting) SystemOnly() bool {
return s.setting.SystemOnly()
}
8 changes: 8 additions & 0 deletions pkg/settings/setting.go
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,9 @@ type Setting interface {
// Reserved settings are still accessible to users, but they don't get
// listed out when retrieving all settings.
Visibility() Visibility

// SystemOnly indicates if a setting is only applicable to the system tenant.
SystemOnly() bool
}

// WritableSetting is the exported interface of non-masked settings.
Expand Down Expand Up @@ -263,6 +266,7 @@ const (
type common struct {
description string
visibility Visibility
systemOnly bool
// Each setting has a slotIdx which is used as a handle with Values.
slotIdx int
nonReportable bool
Expand Down Expand Up @@ -298,6 +302,10 @@ func (i common) Visibility() Visibility {
return i.visibility
}

func (i common) SystemOnly() bool {
return i.systemOnly
}

func (i common) isReportable() bool {
return !i.nonReportable
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/set_cluster_setting.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,11 @@ func (p *planner) SetClusterSetting(
return nil, errors.AssertionFailedf("expected writable setting, got %T", v)
}

if setting.SystemOnly() && !p.execCfg.Codec.ForSystemTenant() {
return nil, pgerror.Newf(pgcode.InsufficientPrivilege,
"setting %s is only setable in the system tenant", name)
}

var value tree.TypedExpr
if n.Value != nil {
// For DEFAULT, let the value reference be nil. That's a RESET in disguise.
Expand Down

0 comments on commit a4fbace

Please sign in to comment.