Skip to content

Commit 9c56d94

Browse files
craig[bot]spilchenjlinder
committed
131422: server/license: Limit enterprise trial license usage to once per cluster r=fqazi,rafiss a=spilchen Previously, clusters could install a second enterprise trial license. To prevent misuse, we are now restricting the use of this license type to only once per cluster. This update adds the necessary tracking mechanism. A new key has been introduced in the KV system key range to record the number of enterprise trial licenses used, ensuring the data persists across restarts. This change will be backported to versions 24.2, 24.1, 23.2, and 23.1. Epic: CRDB-39988 Closes: CRDB-40416 Release note (general change): Attempting to install a second enterprise trial license on the same cluster will now fail. 131709: license-linter: Fix typo r=celiala a=jlinder Missed a blank comment line. Part of RE-658 Release note: none Co-authored-by: Matt Spilchen <[email protected]> Co-authored-by: James H. Linder <[email protected]>
3 parents 48d5e61 + 8aafb3f + 48e33e0 commit 9c56d94

File tree

11 files changed

+233
-25
lines changed

11 files changed

+233
-25
lines changed

Diff for: pkg/ccl/utilccl/BUILD.bazel

+1
Original file line numberDiff line numberDiff line change
@@ -48,6 +48,7 @@ go_test(
4848
"//pkg/sql",
4949
"//pkg/testutils",
5050
"//pkg/testutils/serverutils",
51+
"//pkg/testutils/sqlutils",
5152
"//pkg/util/envutil",
5253
"//pkg/util/leaktest",
5354
"//pkg/util/randutil",

Diff for: pkg/ccl/utilccl/license_check.go

+26-4
Original file line numberDiff line numberDiff line change
@@ -28,14 +28,25 @@ import (
2828
"github.com/cockroachdb/redact"
2929
)
3030

31+
// trialLicenseUsageCount keeps track of the number of times a free trial
32+
// license has already been installed on this cluster.
33+
var trialLicenseUsageCount atomic.Int64
34+
3135
var enterpriseLicense = settings.RegisterStringSetting(
3236
settings.SystemVisible,
3337
"enterprise.license",
3438
"the encoded cluster license",
3539
"",
3640
settings.WithValidateString(
3741
func(sv *settings.Values, s string) error {
38-
_, err := decode(s)
42+
l, err := decode(s)
43+
if err != nil {
44+
return err
45+
}
46+
if l != nil && l.Type == licenseccl.License_Trial && trialLicenseUsageCount.Load() > 0 {
47+
return errors.WithHint(errors.Newf("a trial license has previously been installed on this cluster"),
48+
"Please install a non-trial license to continue")
49+
}
3950
return err
4051
},
4152
),
@@ -315,7 +326,10 @@ func check(l *licenseccl.License, at time.Time, org, feature string, withDetails
315326
func RegisterCallbackOnLicenseChange(
316327
ctx context.Context, st *cluster.Settings, licenseEnforcer *licenseserver.Enforcer,
317328
) {
318-
refreshFunc := func(ctx context.Context) {
329+
// refreshFunc is the function responsible for refreshing the enforcer's state.
330+
// The isChange parameter indicates whether the license is actually being updated,
331+
// as opposed to merely refreshing the current license.
332+
refreshFunc := func(ctx context.Context, isChange bool) {
319333
lic, err := getLicense(st)
320334
if err != nil {
321335
log.Errorf(ctx, "unable to refresh license enforcer for license change: %v", err)
@@ -339,9 +353,17 @@ func RegisterCallbackOnLicenseChange(
339353
}
340354
}
341355
licenseEnforcer.RefreshForLicenseChange(ctx, licenseType, licenseExpiry)
356+
357+
cnt, err := licenseEnforcer.CalculateTrialUsageCount(ctx, licenseType, isChange)
358+
if err != nil {
359+
log.Errorf(ctx, "unable to calculate trial license usage count: %v", err)
360+
return
361+
}
362+
trialLicenseUsageCount.Store(cnt)
342363
}
343364
// Install the hook so that we refresh license details when the license changes.
344-
enterpriseLicense.SetOnChange(&st.SV, refreshFunc)
365+
enterpriseLicense.SetOnChange(&st.SV,
366+
func(ctx context.Context) { refreshFunc(ctx, true /* isChange */) })
345367
// Call the refresh function for the current license.
346-
refreshFunc(ctx)
368+
refreshFunc(ctx, false /* isChange */)
347369
}

Diff for: pkg/ccl/utilccl/license_check_test.go

+61-11
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import (
2323
"github.com/cockroachdb/cockroach/pkg/sql"
2424
"github.com/cockroachdb/cockroach/pkg/testutils"
2525
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
26+
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
2627
"github.com/cockroachdb/cockroach/pkg/util/envutil"
2728
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
2829
"github.com/cockroachdb/cockroach/pkg/util/stop"
@@ -273,12 +274,14 @@ func TestRefreshLicenseEnforcerOnLicenseChange(t *testing.T) {
273274
ts1 := timeutil.Unix(1724329716, 0)
274275

275276
ctx := context.Background()
276-
srv := serverutils.StartServerOnly(t, base.TestServerArgs{
277+
srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{
277278
// We are changing a cluster setting that can only be done at the system tenant.
278279
DefaultTestTenant: base.TestIsSpecificToStorageLayerAndNeedsASystemTenant,
279280
Knobs: base.TestingKnobs{
280281
Server: &server.TestingKnobs{
281282
LicenseTestingKnobs: license.TestingKnobs{
283+
Enable: true,
284+
SkipDisable: true,
282285
OverrideStartTime: &ts1,
283286
},
284287
},
@@ -298,36 +301,83 @@ func TestRefreshLicenseEnforcerOnLicenseChange(t *testing.T) {
298301
require.Equal(t, false, enforcer.GetHasLicense())
299302
gracePeriodTS, hasGracePeriod := enforcer.GetGracePeriodEndTS()
300303
require.True(t, hasGracePeriod)
301-
require.Equal(t, ts1.Add(7*24*time.Hour), gracePeriodTS)
304+
require.Equal(t, ts1.Add(30*24*time.Hour), gracePeriodTS)
302305

303306
jan1st2000 := timeutil.Unix(946728000, 0)
304307

305308
for i, tc := range []struct {
306-
license string
309+
// licenses is a list of licenses to be installed sequentially.
310+
// All licenses except the last one must be installed successfully.
311+
// The outcome of the final license is controlled by the errRE field.
312+
licenses []string
313+
// errRE specifies a regular expression that matches the expected error message
314+
// when installing the last license. If no error is expected, this should be
315+
// set to an empty string.
316+
errRE string
317+
// expectedGracePeriodEnd is the timestamp representing when the grace period
318+
// should end. This value is verified only if the last license is installed
319+
// successfully.
307320
expectedGracePeriodEnd time.Time
308321
}{
309322
// Note: all licenses below expire on Jan 1st 2000
310323
//
311324
// Free license - 30 days grace period
312-
{"crl-0-EMDYt8MDGAMiDkNSREIgVW5pdCBUZXN0", jan1st2000.Add(30 * 24 * time.Hour)},
325+
{[]string{"crl-0-EMDYt8MDGAMiDkNSREIgVW5pdCBUZXN0"}, "", jan1st2000.Add(30 * 24 * time.Hour)},
313326
// Trial license - 7 days grace period
314-
{"crl-0-EMDYt8MDGAQiDkNSREIgVW5pdCBUZXN0", jan1st2000.Add(7 * 24 * time.Hour)},
327+
{[]string{"crl-0-EMDYt8MDGAQiDkNSREIgVW5pdCBUZXN0"}, "", jan1st2000.Add(7 * 24 * time.Hour)},
315328
// Enterprise - no grace period
316-
{"crl-0-EMDYt8MDGAEiDkNSREIgVW5pdCBUZXN0KAM", timeutil.UnixEpoch},
329+
{[]string{"crl-0-EMDYt8MDGAEiDkNSREIgVW5pdCBUZXN0KAM"}, "", timeutil.UnixEpoch},
317330
// No license - 7 days grace period
318-
{"", ts1.Add(7 * 24 * time.Hour)},
331+
{[]string{""}, "", ts1.Add(30 * 24 * time.Hour)},
332+
// Only 1 trial license allowed
333+
{[]string{"crl-0-EMDYt8MDGAQiDkNSREIgVW5pdCBUZXN0", "crl-0-EMDYt8MDGAQiDkNSREIgVW5pdCBUZXN0"},
334+
"a trial license has previously been installed on this cluster", timeutil.UnixEpoch},
319335
} {
320336
t.Run(fmt.Sprintf("test %d", i), func(t *testing.T) {
321-
_, err := srv.SQLConn(t).Exec(
322-
fmt.Sprintf("SET CLUSTER SETTING enterprise.license = '%s'", tc.license),
323-
)
337+
// Reset from prior test unit.
338+
cnt, err := enforcer.SetTrialUsageCount(ctx, 0, false /* checkOldCount */)
324339
require.NoError(t, err)
340+
require.Equal(t, int64(0), cnt)
341+
342+
tdb := sqlutils.MakeSQLRunner(sqlDB)
343+
344+
// Loop through all but the last license. They should all succeed.
345+
for i := 0; i < len(tc.licenses)-1; i++ {
346+
sql := fmt.Sprintf("SET CLUSTER SETTING enterprise.license = '%s'", tc.licenses[i])
347+
tdb.Exec(t, sql)
348+
349+
// If installing a trial license, we need to wait for the callback to
350+
// bump the count before continuing. We depend on the count to cause an
351+
// error if another trial license is installed.
352+
l, err := decode(tc.licenses[i])
353+
require.NoError(t, err)
354+
if l.Type == licenseccl.License_Trial {
355+
var cnt int64
356+
require.Eventually(t, func() bool {
357+
cnt = trialLicenseUsageCount.Load()
358+
return cnt > 0
359+
}, 20*time.Second, time.Millisecond,
360+
"trialLicenseUsageCount last returned %t", cnt)
361+
}
362+
}
363+
364+
// Handle the last license separately
365+
lastLicense := tc.licenses[len(tc.licenses)-1]
366+
sql := fmt.Sprintf("SET CLUSTER SETTING enterprise.license = '%s'", lastLicense)
367+
368+
if tc.errRE != "" {
369+
tdb.ExpectErr(t, tc.errRE, sql) // The last license may result in an error
370+
return
371+
}
372+
373+
tdb.Exec(t, sql)
374+
325375
// The SQL can return back before the callback has finished. So, we wait a
326376
// bit to see if the desired state is reached.
327377
var hasLicense bool
328378
require.Eventually(t, func() bool {
329379
hasLicense = enforcer.GetHasLicense()
330-
return (tc.license != "") == hasLicense
380+
return (lastLicense != "") == hasLicense
331381
}, 20*time.Second, time.Millisecond,
332382
"GetHasLicense() last returned %t", hasLicense)
333383
var ts time.Time

Diff for: pkg/gen/stringer.bzl

+1
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@ STRINGER_SRCS = [
1515
"//pkg/multitenant/tenantcapabilities:id_string.go",
1616
"//pkg/raft/quorum:voteresult_string.go",
1717
"//pkg/roachpb:leasetype_string.go",
18+
"//pkg/server/license:lictype_string.go",
1819
"//pkg/sql/catalog/catalogkeys:commenttype_string.go",
1920
"//pkg/sql/catalog/catpb:privilegedescversion_string.go",
2021
"//pkg/sql/catalog/descpb:formatversion_string.go",

Diff for: pkg/keys/constants.go

+3
Original file line numberDiff line numberDiff line change
@@ -305,6 +305,9 @@ var (
305305
// set during cluster initialization, by which a license must be installed to avoid
306306
// throttling. The value is stored as the number of seconds since the Unix epoch.
307307
ClusterInitGracePeriodTimestamp = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("lic-gpi-ts")))
308+
// TrialLicenseUsageCount is used to keep track of the number of times a trial
309+
// license was installed on the cluster.
310+
TrialLicenseUsageCount = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("lic-tluc")))
308311
//
309312
// NodeIDGenerator is the global node ID generator sequence.
310313
NodeIDGenerator = roachpb.Key(makeKey(SystemPrefix, roachpb.RKey("node-idgen")))

Diff for: pkg/keys/doc.go

+1
Original file line numberDiff line numberDiff line change
@@ -249,6 +249,7 @@ var _ = [...]interface{}{
249249
NodeLivenessPrefix, // "\x00liveness-"
250250
BootstrapVersionKey, // "bootstrap-version"
251251
ClusterInitGracePeriodTimestamp, // "lic-gpi-ts"
252+
TrialLicenseUsageCount, // "lic-tluc"
252253
NodeIDGenerator, // "node-idgen"
253254
RangeIDGenerator, // "range-idgen"
254255
StatusPrefix, // "status-"

Diff for: pkg/server/license/BUILD.bazel

+9
Original file line numberDiff line numberDiff line change
@@ -1,11 +1,13 @@
11
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
2+
load("//build:STRINGER.bzl", "stringer")
23

34
go_library(
45
name = "license",
56
srcs = [
67
"cclbridge.go",
78
"enforcer.go",
89
"opts.go",
10+
":gen-lictype-stringer", # keep
911
],
1012
importpath = "github.com/cockroachdb/cockroach/pkg/server/license",
1113
visibility = ["//visibility:public"],
@@ -50,3 +52,10 @@ go_test(
5052
"@com_github_stretchr_testify//require",
5153
],
5254
)
55+
56+
stringer(
57+
name = "gen-lictype-stringer",
58+
src = "cclbridge.go",
59+
additional_args = ["-linecomment"],
60+
typ = "LicType",
61+
)

Diff for: pkg/server/license/cclbridge.go

+6-5
Original file line numberDiff line numberDiff line change
@@ -29,10 +29,11 @@ var RegisterCallbackOnLicenseChange = func(context.Context, *cluster.Settings, *
2929
// enforcer.
3030
type LicType int
3131

32+
//go:generate stringer -type=LicType -linecomment
3233
const (
33-
LicTypeNone LicType = iota
34-
LicTypeTrial
35-
LicTypeFree
36-
LicTypeEnterprise
37-
LicTypeEvaluation
34+
LicTypeNone LicType = iota // none
35+
LicTypeTrial // trial
36+
LicTypeFree // free
37+
LicTypeEnterprise // enterprise
38+
LicTypeEvaluation // evaluation
3839
)

Diff for: pkg/server/license/enforcer.go

+91-5
Original file line numberDiff line numberDiff line change
@@ -87,6 +87,14 @@ type Enforcer struct {
8787
// When enabled, all checks, including telemetry and expired license validation,
8888
// are bypassed. This is typically used to disable enforcement for single-node deployments.
8989
isDisabled atomic.Bool
90+
91+
// trialUsageCount keeps track of the number of times a free trial license has
92+
// been used on this cluster.
93+
trialUsageCount atomic.Int64
94+
95+
// db is a pointer to the database for use for KV read/writes. This is only
96+
// set for the system tenant.
97+
db isql.DB
9098
}
9199

92100
type TestingKnobs struct {
@@ -181,7 +189,7 @@ func (e *Enforcer) Start(ctx context.Context, st *cluster.Settings, opts ...Opti
181189
e.maybeLogActiveOverrides(ctx)
182190

183191
if !startDisabled {
184-
if err := e.maybeWriteClusterInitGracePeriodTS(ctx, options); err != nil {
192+
if err := e.readClusterMetadata(ctx, options); err != nil {
185193
return err
186194
}
187195
}
@@ -204,9 +212,9 @@ func (e *Enforcer) Start(ctx context.Context, st *cluster.Settings, opts ...Opti
204212
return nil
205213
}
206214

207-
// maybeWriteClusterInitGracePeriodTS checks if the cluster init grace period
208-
// timestamp needs to be written to the KV layer and writes it if needed.
209-
func (e *Enforcer) maybeWriteClusterInitGracePeriodTS(ctx context.Context, options options) error {
215+
// readClusterMetadata will read, and maybe write, cluster metadata for license
216+
// enforcement. The metadata is stored in the KV system keyspace.
217+
func (e *Enforcer) readClusterMetadata(ctx context.Context, options options) error {
210218
// Secondary tenants do not have access to the system keyspace where
211219
// the cluster init grace period is stored. As a fallback, we apply a 7-day
212220
// grace period from the tenant's start time, which is used only when no
@@ -223,7 +231,26 @@ func (e *Enforcer) maybeWriteClusterInitGracePeriodTS(ctx context.Context, optio
223231
return nil
224232
}
225233

234+
// Save a pointer to the database for future use in updating the trial license
235+
// usage count. This is only set for the system tenant.
236+
e.db = options.db
237+
226238
return options.db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error {
239+
// Cache the current trial usage count.
240+
trialUsageCount, err := txn.KV().Get(ctx, keys.TrialLicenseUsageCount)
241+
if err != nil {
242+
return err
243+
}
244+
if trialUsageCount.Value == nil {
245+
e.trialUsageCount.Store(0)
246+
} else {
247+
e.trialUsageCount.Store(trialUsageCount.ValueInt())
248+
}
249+
log.Infof(ctx, "trial license usage count initialized to %d", e.trialUsageCount.Load())
250+
251+
// Cache and maybe set the cluster init grace period timestamp. This is the
252+
// grace period we will use if the cluster does not have a license installed.
253+
//
227254
// We could use a conditional put for this logic. However, we want to read
228255
// and cache the value, and the common case is that the value will be read.
229256
// Only during the initialization of the first node in the cluster will we
@@ -433,7 +460,7 @@ func (e *Enforcer) RefreshForLicenseChange(
433460
}
434461

435462
var sb strings.Builder
436-
sb.WriteString("enforcer license updated: ")
463+
sb.WriteString(fmt.Sprintf("enforcer license updated: type is %s, ", licType.String()))
437464
gpEnd, _ := e.GetGracePeriodEndTS()
438465
if !gpEnd.IsZero() {
439466
sb.WriteString(fmt.Sprintf("grace period ends at %q, ", gpEnd))
@@ -446,6 +473,65 @@ func (e *Enforcer) RefreshForLicenseChange(
446473
log.Infof(ctx, "%s", sb.String())
447474
}
448475

476+
// CalculateTrialUsageCount returns the number of times a trial license has
477+
// been used, including the current trial license if a new one is being applied.
478+
// This function increments the count if the current license is changing and is a trial.
479+
func (e *Enforcer) CalculateTrialUsageCount(
480+
ctx context.Context, currentLicense LicType, isLicenseChange bool,
481+
) (int64, error) {
482+
// If we aren't setting a new trial license, return the cached copy. The e.db
483+
// check is necessary as that's needed to read/write to the KV. This will be
484+
// set for the system tenant, which is where the license can ever be set anyway.
485+
if currentLicense != LicTypeTrial || !isLicenseChange || e.db == nil {
486+
return e.trialUsageCount.Load(), nil
487+
}
488+
489+
return e.SetTrialUsageCount(ctx, e.trialUsageCount.Load()+1, true)
490+
}
491+
492+
// SetTrialUsageCount is an API to set the trial usage count in the enforcer and
493+
// in the KV. The value in the enforcer is always updated. If checkOldCount is
494+
// true, the update to the KV is conditional on the old value matching trialUsageCount.
495+
func (e *Enforcer) SetTrialUsageCount(
496+
ctx context.Context, newCount int64, checkOldCount bool,
497+
) (cnt int64, err error) {
498+
if e.db == nil {
499+
return 0, errors.AssertionFailedf("no database set")
500+
}
501+
err = e.db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error {
502+
// If checking the old count, we only do the update in the KV if the
503+
// existing value for trialUsageCount matches what's in the KV already
504+
// (a missing key is treated as 0). This is necessary to ensure a license
505+
// change to use the trial license will only increase the count by 1 when
506+
// this function is called for each node.
507+
if checkOldCount {
508+
oldVal, err := txn.KV().Get(ctx, keys.TrialLicenseUsageCount)
509+
if err != nil {
510+
return err
511+
}
512+
if oldVal.Value == nil && e.trialUsageCount.Load() != 0 {
513+
e.trialUsageCount.Store(0)
514+
return nil
515+
} else if oldVal.Value != nil && oldVal.ValueInt() != e.trialUsageCount.Load() {
516+
e.trialUsageCount.Store(oldVal.ValueInt())
517+
return nil
518+
}
519+
}
520+
err = txn.KV().Put(ctx, keys.TrialLicenseUsageCount, newCount)
521+
if err != nil {
522+
return err
523+
}
524+
e.trialUsageCount.Store(newCount)
525+
return nil
526+
})
527+
if err != nil {
528+
return
529+
}
530+
cnt = e.trialUsageCount.Load()
531+
log.Infof(ctx, "trial license usage count is %d", cnt)
532+
return
533+
}
534+
449535
// Disable turns off all license enforcement for the lifetime of this object.
450536
func (e *Enforcer) Disable(ctx context.Context) {
451537
// We provide an override so that we can continue to test license enforcement

0 commit comments

Comments
 (0)