diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index c1809384158f..ced6ab6d04f4 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -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 diff --git a/pkg/server/license/BUILD.bazel b/pkg/server/license/BUILD.bazel index aa86a16c3b5a..68c41e26adc4 100644 --- a/pkg/server/license/BUILD.bazel +++ b/pkg/server/license/BUILD.bazel @@ -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", diff --git a/pkg/server/license/enforcer.go b/pkg/server/license/enforcer.go index 84611ddbb2f8..ff14e17fee8a 100644 --- a/pkg/server/license/enforcer.go +++ b/pkg/server/license/enforcer.go @@ -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" @@ -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, @@ -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 @@ -305,7 +301,13 @@ 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() { @@ -313,7 +315,17 @@ func (e *Enforcer) MaybeFailIfThrottled(ctx context.Context, txnsOpened int64) ( } 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.", @@ -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 } @@ -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() @@ -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) +} diff --git a/pkg/server/license/enforcer_test.go b/pkg/server/license/enforcer_test.go index 3cd66f782e6c..e70c9417b35d 100644 --- a/pkg/server/license/enforcer_test.go +++ b/pkg/server/license/enforcer_test.go @@ -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) @@ -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{} @@ -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 { @@ -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) + } }) } } diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 22f4f3e31725..cb17230286be 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -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) } }