Skip to content

Commit

Permalink
logcrash, ccl: fail if setting incompatible license and reporting cfg
Browse files Browse the repository at this point in the history
With the rollout of the license requirements with CRDB, we are going to
begin disallowing the disabling of diagnostics under specific license
types. If the caller is to disable diagnostics, they must have a license
which is not Free or Trial. Conversely, if the caller is to set a Free
or Trial license, they must already be submitting diagnostics
information for it to succeed.

Epic: CRDB-40209
Fixes: CRDB-41232

Release note (sql change): cluster settings enterprise.license and
diagnostics.reporting.enabled have additional validation
  • Loading branch information
angles-n-daemons committed Sep 20, 2024
1 parent 29ad4d3 commit 1f4ae72
Show file tree
Hide file tree
Showing 9 changed files with 245 additions and 12 deletions.
34 changes: 32 additions & 2 deletions pkg/ccl/utilccl/license_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@ package utilccl

import (
"context"
"fmt"
"strconv"
"strings"
"sync/atomic"
"time"
Expand All @@ -35,8 +37,36 @@ var enterpriseLicense = settings.RegisterStringSetting(
"",
settings.WithValidateString(
func(sv *settings.Values, s string) error {
_, err := decode(s)
return err
// lookup diagnostic reporting setting
reportingSetting, ok, _ := settings.LookupForLocalAccess("diagnostics.reporting.enabled", true /* forSystemTenant */)
if !ok {
return fmt.Errorf("unable to find setting for diagnostic reporting")
}
reportingStr, err := reportingSetting.DecodeToString(reportingSetting.Encoded(sv))
if err != nil {
return err
}

reporting, err := strconv.ParseBool(reportingStr)
if err != nil {
return err
}

// decode license
license, err := decode(s)
if err != nil {
return err
}
if license == nil {
return nil
}

// if the license is limited and reporting is disabled, do not allow it to be set
isLimited := license.Type == licenseccl.License_Free || license.Type == licenseccl.License_Trial
if !reporting && isLimited {
return errors.New("diagnostics.reporting.enabled must be true to use this license")
}
return nil
},
),
// Even though string settings are non-reportable by default, we
Expand Down
84 changes: 84 additions & 0 deletions pkg/ccl/utilccl/license_check_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ package utilccl
import (
"context"
"fmt"
"strconv"
"testing"
"time"

Expand Down Expand Up @@ -69,6 +70,82 @@ func TestSettingAndCheckingLicense(t *testing.T) {
}
}

// test setting a license with a specific diagnostics setting
func TestSetLicenseWithDiagnosticsReporting(t *testing.T) {
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
t0 := timeutil.Unix(0, 0)

for _, tc := range []struct {
lit licenseccl.License_Type
diagnostics bool
err string
}{
{licenseccl.License_Free, false, "unable to disable diagnostics with license type Free"},
{licenseccl.License_Free, true, ""},
{licenseccl.License_Trial, false, "unable to disable diagnostics with license type Trial"},
{licenseccl.License_Trial, true, ""},
{licenseccl.License_NonCommercial, false, ""},
{licenseccl.License_NonCommercial, true, ""},
{licenseccl.License_Enterprise, false, ""},
{licenseccl.License_Enterprise, true, ""},
{licenseccl.License_Evaluation, false, ""},
{licenseccl.License_Evaluation, true, ""},
} {
lic, _ := (&licenseccl.License{
Type: tc.lit,
ValidUntilUnixSec: t0.AddDate(0, 1, 0).Unix(),
}).Encode()
updater := st.MakeUpdater()
if err := setDiagnosticsReporting(ctx, updater, tc.diagnostics); err != nil {
t.Fatal(err)
}
if err := setLicense(ctx, updater, lic); !testutils.IsError(
err, tc.err,
) {
t.Fatalf("%s %t: expected err %q, got %v", tc.lit, tc.diagnostics, tc.err, err)
}

}
}

// test setting the diagnostics setting with a specific license (reverse of the above)
func TestSetDiagnosticsReportingWithLicense(t *testing.T) {
ctx := context.Background()
st := cluster.MakeTestingClusterSettings()
t0 := timeutil.Unix(0, 0)
for _, tc := range []struct {
lit licenseccl.License_Type
diagnostics bool
err string
}{
{licenseccl.License_Free, false, "unable to disable diagnostics with license type Free"},
{licenseccl.License_Free, true, ""},
{licenseccl.License_Trial, false, "unable to disable diagnostics with license type Trial"},
{licenseccl.License_Trial, true, ""},
{licenseccl.License_NonCommercial, false, ""},
{licenseccl.License_NonCommercial, true, ""},
{licenseccl.License_Enterprise, false, ""},
{licenseccl.License_Enterprise, true, ""},
{licenseccl.License_Evaluation, false, ""},
{licenseccl.License_Evaluation, true, ""},
} {
lic, _ := (&licenseccl.License{
Type: tc.lit,
ValidUntilUnixSec: t0.AddDate(0, 1, 0).Unix(),
}).Encode()
updater := st.MakeUpdater()
if err := setLicense(ctx, updater, lic); err != nil {
t.Fatal(err)
}
if err := setDiagnosticsReporting(ctx, updater, tc.diagnostics); !testutils.IsError(
err, tc.err,
) {
t.Fatalf("%s %t: expected err %q, got %v", tc.lit, tc.diagnostics, tc.err, err)
}
}
}

func TestGetLicenseTypePresent(t *testing.T) {
defer leaktest.AfterTest(t)()

Expand Down Expand Up @@ -266,6 +343,13 @@ func setLicense(ctx context.Context, updater settings.Updater, val string) error
})
}

func setDiagnosticsReporting(ctx context.Context, updater settings.Updater, val bool) error {
return updater.Set(ctx, "diagnostics.reporting.enabled", settings.EncodedValue{
Value: strconv.FormatBool(val),
Type: "b",
})
}

func TestRefreshLicenseEnforcerOnLicenseChange(t *testing.T) {
defer leaktest.AfterTest(t)()

Expand Down
1 change: 0 additions & 1 deletion pkg/cmd/cockroach-oss/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ disallowed_imports_test(
"cockroach-oss",
disallowed_list = [],
disallowed_prefixes = [
"pkg/ccl",
"pkg/ui/distccl",
],
)
57 changes: 50 additions & 7 deletions pkg/settings/bool.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,8 @@ package settings
import (
"context"
"strconv"

"github.com/cockroachdb/errors"
)

// BoolSetting is the interface of a setting variable that will be
Expand All @@ -21,6 +23,7 @@ import (
type BoolSetting struct {
common
defaultValue bool
validateFn func(*Values, bool) error
}

var _ internalSetting = &BoolSetting{}
Expand Down Expand Up @@ -82,11 +85,29 @@ var _ = (*BoolSetting).Default
// For testing usage only.
func (b *BoolSetting) Override(ctx context.Context, sv *Values, v bool) {
sv.setValueOrigin(ctx, b.slot, OriginOverride)
b.set(ctx, sv, v)
b.setOnValues(ctx, sv, v)
sv.setDefaultOverride(b.slot, v)
}

func (b *BoolSetting) set(ctx context.Context, sv *Values, v bool) {
// Validate that a value conforms with the validation function.
func (b *BoolSetting) Validate(sv *Values, v bool) error {
if b.validateFn != nil {
if err := b.validateFn(sv, v); err != nil {
return err
}
}
return nil
}

func (b *BoolSetting) set(ctx context.Context, sv *Values, v bool) error {
if err := b.Validate(sv, v); err != nil {
return err
}
b.setOnValues(ctx, sv, v)
return nil
}

func (b *BoolSetting) setOnValues(ctx context.Context, sv *Values, v bool) {
vInt := int64(0)
if v {
vInt = 1
Expand All @@ -99,8 +120,7 @@ func (b *BoolSetting) decodeAndSet(ctx context.Context, sv *Values, encoded stri
if err != nil {
return err
}
b.set(ctx, sv, v)
return nil
return b.set(ctx, sv, v)
}

func (b *BoolSetting) decodeAndSetDefaultOverride(
Expand All @@ -117,17 +137,40 @@ func (b *BoolSetting) decodeAndSetDefaultOverride(
func (b *BoolSetting) setToDefault(ctx context.Context, sv *Values) {
// See if the default value was overridden.
if val := sv.getDefaultOverride(b.slot); val != nil {
b.set(ctx, sv, val.(bool))
// As per the semantics of override, these values don't go through
// validation.
_ = b.set(ctx, sv, val.(bool))
return
}
b.set(ctx, sv, b.defaultValue)
if err := b.set(ctx, sv, b.defaultValue); err != nil {
panic(err)
}
}

// RegisterBoolSetting defines a new setting with type bool.
func RegisterBoolSetting(
class Class, key InternalKey, desc string, defaultValue bool, opts ...SettingOption,
) *BoolSetting {
setting := &BoolSetting{defaultValue: defaultValue}
validateFn := func(sv *Values, val bool) error {
for _, opt := range opts {
switch {
case opt.commonOpt != nil:
continue
case opt.validateBoolFn != nil:
default:
panic(errors.AssertionFailedf("wrong validator type"))
}
if err := opt.validateBoolFn(sv, val); err != nil {
return err
}
}
return nil
}
// what to put here?
if err := validateFn(&Values{}, defaultValue); err != nil {
panic(errors.Wrap(err, "invalid default"))
}
setting := &BoolSetting{defaultValue: defaultValue, validateFn: validateFn}
register(class, key, desc, setting)
setting.apply(opts)
return setting
Expand Down
6 changes: 6 additions & 0 deletions pkg/settings/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
// SettingOption is the type of an option that can be passed to Register.
type SettingOption struct {
commonOpt func(*common)
validateBoolFn func(*Values, bool) error
validateDurationFn func(time.Duration) error
validateInt64Fn func(int64) error
validateFloat64Fn func(float64) error
Expand Down Expand Up @@ -108,6 +109,11 @@ func WithValidateFloat(fn func(float64) error) SettingOption {
return SettingOption{validateFloat64Fn: fn}
}

// WithValidateBool adds a validation function for a boolean setting.
func WithValidateBool(fn func(*Values, bool) error) SettingOption {
return SettingOption{validateBoolFn: fn}
}

// WithValidateString adds a validation function for a string setting.
func WithValidateString(fn func(*Values, string) error) SettingOption {
return SettingOption{validateStringFn: fn}
Expand Down
30 changes: 30 additions & 0 deletions pkg/settings/validation_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,22 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils"
)

var cantBeTrue = settings.WithValidateBool(func(sv *settings.Values, b bool) error {
fmt.Println("testing it cant be true")
if b {
return fmt.Errorf("it cant be true")
}
return nil
})

var cantBeFalse = settings.WithValidateBool(func(sv *settings.Values, b bool) error {
fmt.Println("testing it cant be false")
if !b {
return fmt.Errorf("it cant be false")
}
return nil
})

func TestValidationOptions(t *testing.T) {
type subTest struct {
val interface{}
Expand Down Expand Up @@ -182,6 +198,20 @@ func TestValidationOptions(t *testing.T) {
{val: 11, opt: settings.ByteSizeWithMinimum(10), expectedErr: ""},
},
},
{
testLabel: "bool",
settingFn: func(n int, bval interface{}, opt settings.SettingOption) settings.Setting {
val := bval.(bool)
return settings.RegisterBoolSetting(settings.SystemOnly, settings.InternalKey(fmt.Sprintf("test-%d", n)), "desc",
val, opt)
},
subTests: []subTest{
{val: true, opt: cantBeTrue, expectedErr: "it cant be true"},
{val: false, opt: cantBeTrue, expectedErr: ""},
{val: true, opt: cantBeFalse, expectedErr: ""},
{val: false, opt: cantBeFalse, expectedErr: "it cant be false"},
},
},
}

for _, tc := range testCases {
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/set_cluster_setting.go
Original file line number Diff line number Diff line change
Expand Up @@ -782,6 +782,9 @@ func toSettingString(
return "", errors.Errorf("cannot use %s %T value for string setting", d.ResolvedType(), d)
case *settings.BoolSetting:
if b, ok := d.(*tree.DBool); ok {
if err := setting.Validate(&st.SV, bool(*b)); err != nil {
return "", err
}
return settings.EncodeBool(bool(*b)), nil
}
return "", errors.Errorf("cannot use %s %T value for bool setting", d.ResolvedType(), d)
Expand Down
1 change: 1 addition & 0 deletions pkg/util/log/logcrash/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ go_library(
},
deps = [
"//pkg/build",
"//pkg/ccl/utilccl/licenseccl",
"//pkg/settings",
"//pkg/util/envutil",
"//pkg/util/log",
Expand Down
Loading

0 comments on commit 1f4ae72

Please sign in to comment.