Skip to content

Commit

Permalink
Merge #131284
Browse files Browse the repository at this point in the history
131284: server/license: Add notice for upcoming throttling r=fqazi a=spilchen

Previously, throttling errors would only occur when active throttling was in effect. This update adds notices to alert clients when throttling is imminent, without disrupting running queries. A notice will be returned in the following cases:
- If no license is installed or the license has expired, but we are still within the grace period.
- If the license requires telemetry and no telemetry data has been received in the past 3 days.

In both cases, a notice is triggered only if the max limit of 5 concurrent open transactions is exceeded.

Additionally, notices are logged to cockroach.log at most once every 5 minutes. This commit also cleans up prior logging code by utilizing log.Every() for more efficient log gating.

This will be backported to 24.2, 24.1, 23.2 and 23.1.

Epic: CRDB-39988
Closes CRDB-39992
Release note: None

Co-authored-by: Matt Spilchen <[email protected]>
  • Loading branch information
craig[bot] and spilchen committed Sep 24, 2024
2 parents 56e2eb3 + 2975eeb commit 090609b
Show file tree
Hide file tree
Showing 5 changed files with 111 additions and 54 deletions.
1 change: 1 addition & 0 deletions .github/CODEOWNERS
Validating CODEOWNERS rules …
Original file line number Diff line number Diff line change
Expand Up @@ -200,6 +200,7 @@
/pkg/server/init*.go @cockroachdb/kv-prs @cockroachdb/server-prs
/pkg/server/intent_*.go @cockroachdb/kv-prs @cockroachdb/server-prs
/pkg/server/key_vis* @cockroachdb/obs-prs
/pkg/server/license/ @cockroachdb/sql-foundations
/pkg/server/load_endpoint* @cockroachdb/obs-prs @cockroachdb/server-prs
/pkg/server/loss_of_quorum*.go @cockroachdb/kv-prs
/pkg/server/migration* @cockroachdb/sql-foundations
Expand Down
1 change: 1 addition & 0 deletions pkg/server/license/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ go_library(
"//pkg/sql/isql",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/pgwire/pgnotice",
"//pkg/sql/sem/tree",
"//pkg/util/envutil",
"//pkg/util/log",
Expand Down
111 changes: 77 additions & 34 deletions pkg/server/license/enforcer.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -75,15 +76,9 @@ type Enforcer struct {
// hasLicense is true if any license is installed.
hasLicense atomic.Bool

// lastLicenseThrottlingLogTime keeps track of the last time we logged a
// message because we had to throttle due to a license issue. The value
// stored is the number of seconds since the unix epoch.
lastLicenseThrottlingLogTime atomic.Int64

// lastTelemetryThrottlingLogTime keeps track of the last time we logged a
// message because we had to throttle due to a telemetry issue. The value
// stored is the number of seconds since the unix epoch.
lastTelemetryThrottlingLogTime atomic.Int64
// throttleLogger is a logger for throttle-related messages. It is used to
// emit logs only every few minutes to avoid spamming the logs.
throttleLogger log.EveryN

// isDisabled is a global override that completely disables license enforcement.
// When enabled, all checks, including telemetry and expired license validation,
Expand Down Expand Up @@ -147,7 +142,8 @@ func GetEnforcerInstance() *Enforcer {
// newEnforcer creates a new Enforcer object.
func newEnforcer() *Enforcer {
e := &Enforcer{
startTime: timeutil.Now(),
startTime: timeutil.Now(),
throttleLogger: log.Every(5 * time.Minute),
}
e.isDisabled.Store(true) // Start disabled until Start() is called
return e
Expand Down Expand Up @@ -305,15 +301,31 @@ func (e *Enforcer) GetTelemetryDeadline() (deadline, lastPing time.Time, ok bool
// returning an error if throttling conditions are met. Throttling may be triggered
// if the maximum number of open transactions is exceeded and the grace period has
// ended or if required diagnostic reporting has not been received.
func (e *Enforcer) MaybeFailIfThrottled(ctx context.Context, txnsOpened int64) (err error) {
//
// If throttling hasn't occurred yet but may soon, a notice is returned for the client.
// The notice is only returned if no throttling is detected. Callers should always
// check the error first.
func (e *Enforcer) MaybeFailIfThrottled(
ctx context.Context, txnsOpened int64,
) (notice pgnotice.Notice, err error) {
// Early out if the number of transactions is below the max allowed or
// everything has been disabled.
if txnsOpened <= e.getMaxOpenTransactions() || e.isDisabled.Load() {
return
}

now := e.getThrottleCheckTS()
if gracePeriodEnd, ok := e.GetGracePeriodEndTS(); ok && now.After(gracePeriodEnd) {
expiryTS, hasExpiry := e.getLicenseExpiryTS()

// If the license doesn't require telemetry and hasn't expired,
// we can exit without any further checks.
if !e.licenseRequiresTelemetry.Load() && hasExpiry && expiryTS.After(now) {
return
}

// Throttle if the license has expired, is missing, and the grace period has ended.
gracePeriodEnd, hasGracePeriod := e.GetGracePeriodEndTS()
if hasGracePeriod && now.After(gracePeriodEnd) {
if e.GetHasLicense() {
err = errors.WithHintf(pgerror.Newf(pgcode.CCLValidLicenseRequired,
"License expired on %s. The maximum number of concurrently open transactions has been reached.",
Expand All @@ -324,22 +336,53 @@ func (e *Enforcer) MaybeFailIfThrottled(ctx context.Context, txnsOpened int64) (
"No license installed. The maximum number of concurrently open transactions has been reached."),
"Obtain and install a valid license to continue.")
}
e.maybeLogError(ctx, err, &e.lastLicenseThrottlingLogTime,
fmt.Sprintf(", license expired with a grace period that ended at %s", gracePeriodEnd))
if e.throttleLogger.ShouldLog() {
log.Infof(ctx, "throttling for license enforcement is active, license expired with a grace period "+
"that ended at %s", gracePeriodEnd)
}
return
}

if deadlineTS, lastPingTS, ok := e.GetTelemetryDeadline(); ok && now.After(deadlineTS) {
// For cases where the license requires telemetry, throttle if we are past the deadline
deadlineTS, lastPingTS, requiresTelemetry := e.GetTelemetryDeadline()
if requiresTelemetry && now.After(deadlineTS) {
err = errors.WithHintf(pgerror.Newf(pgcode.CCLValidLicenseRequired,
"The maximum number of concurrently open transactions has been reached because the license requires "+
"diagnostic reporting, but none has been received by Cockroach Labs."),
"Ensure diagnostic reporting is enabled and verify that nothing is blocking network access to the "+
"Cockroach Labs reporting server. You can also consider changing your license to one that doesn't "+
"require diagnostic reporting to be emitted.")
e.maybeLogError(ctx, err, &e.lastTelemetryThrottlingLogTime,
fmt.Sprintf("due to no telemetry data received, last received at %s", lastPingTS))
if e.throttleLogger.ShouldLog() {
log.Infof(ctx, "throttling for license enforcement is active, due to no telemetry data received, "+
"last received at %s", lastPingTS)
}
return
}

// Emit a warning if throttling is imminent. This could mean the license is expired but within
// the grace period, or that required telemetry data hasn't been sent recently.
if hasGracePeriod && now.After(expiryTS) {
if e.GetHasLicense() {
notice = pgnotice.Newf(
"Your license expired on %s. Throttling will begin after %s. Please install a new license to prevent this.",
expiryTS, gracePeriodEnd)
} else {
notice = pgnotice.Newf(
"No license is installed. Throttling will begin after %s unless a license is installed before then.",
gracePeriodEnd)
}
} else if requiresTelemetry && e.addThrottleWarningDelayForNoTelemetry(lastPingTS).Before(now) {
notice = pgnotice.Newf(
"Your license requires diagnostic reporting, but no data has been sent since %s. Throttling will begin "+
"after %s unless the data is sent.",
lastPingTS, deadlineTS)
}
if notice != nil {
if e.throttleLogger.ShouldLog() {
log.Infof(ctx, "throttling will happen soon: %s", notice.Error())
}
}

return
}

Expand Down Expand Up @@ -458,23 +501,6 @@ func (e *Enforcer) getMaxTelemetryInterval() time.Duration {
return newTimeframe
}

// maybeLogError logs a throttling error message if one hasn't been logged
// recently. This helps alert about throttling issues without flooding the
// CockroachDB log. It also serves as a useful breadcrumb for debugging,
// particularly in automated test runs where client responses may not be fully
// examined.
func (e *Enforcer) maybeLogError(
ctx context.Context, err error, lastLogTimestamp *atomic.Int64, additionalMsg string,
) {
nextLogMessage := timeutil.Unix(lastLogTimestamp.Load(), 0).Add(5 * time.Minute)

now := timeutil.Now()
if now.After(nextLogMessage) {
lastLogTimestamp.Store(now.Unix())
log.Infof(ctx, "throttling for license enforcement is active %s: %s", additionalMsg, err.Error())
}
}

// maybeLogActiveOverrides is a debug tool to indicate any env var overrides.
func (e *Enforcer) maybeLogActiveOverrides(ctx context.Context) {
maxOpenTxns := e.getMaxOpenTransactions()
Expand Down Expand Up @@ -544,3 +570,20 @@ func (e *Enforcer) getIsNewClusterEstimate(ctx context.Context, txn isql.Txn) (b
log.Infof(ctx, "cluster init is not within the bounds of the enforcer start time: %v", ts)
return false, nil
}

// getLicenseExpiryTS returns the license expiration timestamp.
// If no license is installed, it returns false for the second parameter.
func (e *Enforcer) getLicenseExpiryTS() (ts time.Time, ok bool) {
if !e.hasLicense.Load() {
return
}
ts = timeutil.Unix(e.licenseExpiryTS.Load(), 0)
ok = true
return
}

// addThrottleWarningDelayForNoTelemetry will return the time when we should start emitting
// a notice that we haven't received telemetry and we will throttle soon.
func (e *Enforcer) addThrottleWarningDelayForNoTelemetry(t time.Time) time.Time {
return t.Add(3 * 24 * time.Hour)
}
48 changes: 29 additions & 19 deletions pkg/server/license/enforcer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,7 @@ func TestThrottle(t *testing.T) {
const AtTxnThreshold = 5
const OverTxnThreshold = 7

t0 := timeutil.Unix(1724884362, 0)
t0 := timeutil.Unix(1724884362, 0) // 08/28/24 10:32:42 PM UTC
t1d := t0.Add(24 * time.Hour)
t8d := t0.Add(8 * 24 * time.Hour)
t10d := t0.Add(10 * 24 * time.Hour)
Expand All @@ -172,41 +172,42 @@ func TestThrottle(t *testing.T) {
licExpiry time.Time
checkTs time.Time
expectedErrRegex string
expectedNoticeRegex string
}{
// Expired free license but under the transaction threshold
{UnderTxnThreshold, license.LicTypeFree, t0, t1d, t8d, t45d, ""},
{UnderTxnThreshold, license.LicTypeFree, t0, t1d, t8d, t45d, "", ""},
// Expired trial license but at the transaction threshold
{AtTxnThreshold, license.LicTypeTrial, t0, t30d, t8d, t45d, ""},
{AtTxnThreshold, license.LicTypeTrial, t0, t30d, t8d, t45d, "", ""},
// Over the transaction threshold but not expired
{OverTxnThreshold, license.LicTypeFree, t0, t10d, t45d, t10d, ""},
{OverTxnThreshold, license.LicTypeFree, t0, t10d, t45d, t10d, "", ""},
// Expired free license, past the grace period
{OverTxnThreshold, license.LicTypeFree, t0, t30d, t10d, t45d, "License expired"},
{OverTxnThreshold, license.LicTypeFree, t0, t30d, t10d, t45d, "License expired", ""},
// Expired free license, but not past the grace period
{OverTxnThreshold, license.LicTypeFree, t0, t30d, t10d, t17d, ""},
{OverTxnThreshold, license.LicTypeFree, t0, t30d, t10d, t17d, "", "license expired.*Throttling will begin"},
// Valid free license, but telemetry ping hasn't been received in 7 days.
{OverTxnThreshold, license.LicTypeFree, t0, t10d, t45d, t17d, ""},
{OverTxnThreshold, license.LicTypeFree, t0, t10d, t45d, t17d, "", "Your license requires diagnostic reporting.*Throttling will begin"},
// Valid free license, but telemetry ping hasn't been received in 8 days.
{OverTxnThreshold, license.LicTypeFree, t0, t10d, t45d, t18d, "diagnostic reporting"},
{OverTxnThreshold, license.LicTypeFree, t0, t10d, t45d, t18d, "diagnostic reporting", ""},
// No license but within grace period still
{OverTxnThreshold, license.LicTypeNone, t0, t0, t0, t1d, ""},
{OverTxnThreshold, license.LicTypeNone, t0, t0, t0, t1d, "", "No license.*Throttling will begin"},
// No license but beyond grace period
{OverTxnThreshold, license.LicTypeNone, t0, t0, t0, t8d, "No license installed"},
{OverTxnThreshold, license.LicTypeNone, t0, t0, t0, t8d, "No license installed", ""},
// Trial license has expired but still within grace period
{OverTxnThreshold, license.LicTypeTrial, t0, t30d, t10d, t15d, ""},
{OverTxnThreshold, license.LicTypeTrial, t0, t30d, t10d, t15d, "", "license expired.*Throttling will begin"},
// Trial license has expired and just at the edge of the grace period.
{OverTxnThreshold, license.LicTypeTrial, t0, t45d, t10d, t17d, ""},
{OverTxnThreshold, license.LicTypeTrial, t0, t45d, t10d, t17d, "", "license expired.*Throttling will begin"},
// Trial license has expired and just beyond the grace period.
{OverTxnThreshold, license.LicTypeTrial, t0, t45d, t10d, t18d, "License expired"},
{OverTxnThreshold, license.LicTypeTrial, t0, t45d, t10d, t18d, "License expired", ""},
// No throttling if past the expiry of an enterprise license
{OverTxnThreshold, license.LicTypeEnterprise, t0, t0, t8d, t46d, ""},
{OverTxnThreshold, license.LicTypeEnterprise, t0, t0, t8d, t46d, "", ""},
// Telemetry isn't needed for enterprise license
{OverTxnThreshold, license.LicTypeEnterprise, t0, t0, t45d, t30d, ""},
{OverTxnThreshold, license.LicTypeEnterprise, t0, t0, t45d, t30d, "", ""},
// Telemetry isn't needed for evaluation license
{OverTxnThreshold, license.LicTypeEvaluation, t0, t0, t45d, t30d, ""},
{OverTxnThreshold, license.LicTypeEvaluation, t0, t0, t45d, t30d, "", ""},
// Evaluation license doesn't throttle if expired but within grace period.
{OverTxnThreshold, license.LicTypeEvaluation, t0, t0, t15d, t30d, ""},
{OverTxnThreshold, license.LicTypeEvaluation, t0, t0, t15d, t30d, "", "license expired.*Throttling will begin"},
// Evaluation license does throttle if expired and beyond grace period.
{OverTxnThreshold, license.LicTypeEvaluation, t0, t0, t15d, t46d, "License expired"},
{OverTxnThreshold, license.LicTypeEvaluation, t0, t0, t15d, t46d, "License expired", ""},
} {
t.Run(fmt.Sprintf("test %d", i), func(t *testing.T) {
e := license.Enforcer{}
Expand All @@ -218,7 +219,7 @@ func TestThrottle(t *testing.T) {
lastPingTime: tc.lastTelemetryPingTime,
})
e.RefreshForLicenseChange(ctx, tc.licType, tc.licExpiry)
err := e.MaybeFailIfThrottled(ctx, tc.openTxnsCount)
notice, err := e.MaybeFailIfThrottled(ctx, tc.openTxnsCount)
if tc.expectedErrRegex == "" {
require.NoError(t, err)
} else {
Expand All @@ -228,6 +229,15 @@ func TestThrottle(t *testing.T) {
require.NotNil(t, match, "Error text %q doesn't match the expected regexp of %q",
err.Error(), tc.expectedErrRegex)
}
if tc.expectedNoticeRegex == "" {
require.NoError(t, notice)
} else {
require.Error(t, notice)
re := regexp.MustCompile(tc.expectedNoticeRegex)
match := re.MatchString(notice.Error())
require.NotNil(t, match, "Notice text %q doesn't match the expected regexp of %q",
notice.Error(), tc.expectedNoticeRegex)
}
})
}
}
Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -506,8 +506,10 @@ func (ex *connExecutor) execStmtInOpenState(

// Enforce license policies. Throttling can occur if there is no valid
// license or if it has expired.
if err := ex.server.cfg.LicenseEnforcer.MaybeFailIfThrottled(ctx, curOpen); err != nil {
if notice, err := ex.server.cfg.LicenseEnforcer.MaybeFailIfThrottled(ctx, curOpen); err != nil {
return makeErrEvent(err)
} else if notice != nil {
res.BufferNotice(notice)
}
}

Expand Down

0 comments on commit 090609b

Please sign in to comment.