From 2f96eeb76e0a47f37b4ead260fc4dc0168c1e538 Mon Sep 17 00:00:00 2001 From: Santamaura Date: Wed, 8 Jun 2022 17:38:38 -0400 Subject: [PATCH 1/2] ui, sql: add cluster.preserve_downgrade_option to prometheus and alerts This patch exposes the cluster.preserve_downgrade_option setting in prometheus (as a unix timestamp) with the name cluster.preserve-downgrade-option.last-updated so that users can check when this value was last updated when finalizing cluster upgrades. If the option is reset the metric will be set to 0. An alert banner on the dashboard will also display if the option has been set for 48 hours or greater indicating the upgrade may have not been finalized. Resolves #78620 Release note (sql change, ui change): Add cluster.preserve_downgrade_option to prometheus as a unix timestamp and to alert banners in order to provide some observability in upgrade finalization. --- pkg/clusterversion/BUILD.bazel | 2 + pkg/clusterversion/setting.go | 40 ++++++++++++ pkg/server/server.go | 5 ++ pkg/ts/catalog/chart_catalog.go | 7 +++ .../db-console/src/redux/alerts.spec.ts | 61 ++++++++++++++++++- .../workspaces/db-console/src/redux/alerts.ts | 60 +++++++++++++++++- 6 files changed, 172 insertions(+), 3 deletions(-) diff --git a/pkg/clusterversion/BUILD.bazel b/pkg/clusterversion/BUILD.bazel index 2f39e1b514c3..8f4df1195598 100644 --- a/pkg/clusterversion/BUILD.bazel +++ b/pkg/clusterversion/BUILD.bazel @@ -21,7 +21,9 @@ go_library( "//pkg/roachpb", "//pkg/settings", "//pkg/util/log", + "//pkg/util/metric", "//pkg/util/protoutil", + "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", "@com_github_kr_pretty//:pretty", diff --git a/pkg/clusterversion/setting.go b/pkg/clusterversion/setting.go index abe9c1437b3a..dc221f075eeb 100644 --- a/pkg/clusterversion/setting.go +++ b/pkg/clusterversion/setting.go @@ -16,7 +16,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) @@ -268,3 +270,41 @@ func registerPreserveDowngradeVersionSetting() *settings.StringSetting { s.SetVisibility(settings.Public) return s } + +var metaPreserveDowngradeLastUpdated = metric.Metadata{ + Name: "cluster.preserve-downgrade-option.last-updated", + Help: "Unix timestamp of last updated time for cluster.preserve_downgrade_option", + Measurement: "Timestamp", + Unit: metric.Unit_TIMESTAMP_SEC, +} + +// preserveDowngradeLastUpdatedMetric is a metric gauge that measures the +// time the cluster.preserve_downgrade_option was last updated. +var preserveDowngradeLastUpdatedMetric = metric.NewGauge(metaPreserveDowngradeLastUpdated) + +// RegisterOnVersionChangeCallback is a callback function that updates the +// cluster.preserve-downgrade-option.last-updated when the +// cluster.preserve_downgrade_option settings is changed. +func RegisterOnVersionChangeCallback(sv *settings.Values) { + preserveDowngradeVersion.SetOnChange(sv, func(ctx context.Context) { + var value int64 + downgrade := preserveDowngradeVersion.Get(sv) + if downgrade != "" { + value = timeutil.Now().Unix() + } + preserveDowngradeLastUpdatedMetric.Update(value) + }) +} + +// Metrics defines the settings tracked in prometheus. +type Metrics struct { + PreserveDowngradeLastUpdated *metric.Gauge +} + +// MakeMetrics is a function that creates the metrics defined in the Metrics +// struct. +func MakeMetrics() Metrics { + return Metrics{ + PreserveDowngradeLastUpdated: preserveDowngradeLastUpdatedMetric, + } +} diff --git a/pkg/server/server.go b/pkg/server/server.go index c9fb727b7ca6..231f1ab59807 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -305,6 +305,11 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { registry.AddMetricStruct(runtimeSampler) registry.AddMetric(base.LicenseTTL) + + clusterVersionMetrics := clusterversion.MakeMetrics() + registry.AddMetricStruct(clusterVersionMetrics) + clusterversion.RegisterOnVersionChangeCallback(&st.SV) + err = base.UpdateMetricOnLicenseChange(ctx, cfg.Settings, base.LicenseTTL, timeutil.DefaultTimeSource{}, stopper) if err != nil { log.Errorf(ctx, "unable to initialize periodic license metric update: %v", err) diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 6d2da17da326..a8a5d3215b43 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -2378,6 +2378,13 @@ var charts = []sectionDescription{ }, AxisLabel: "SQL Statements", }, + { + Title: "Cluster.preserve_downgrade_option Last Updated", + Metrics: []string{ + "cluster.preserve-downgrade-option.last-updated", + }, + AxisLabel: "Last Updated Timestamp", + }, { Title: "Byte I/O", Metrics: []string{ diff --git a/pkg/ui/workspaces/db-console/src/redux/alerts.spec.ts b/pkg/ui/workspaces/db-console/src/redux/alerts.spec.ts index f9cf59a7c948..e371e4b4e830 100644 --- a/pkg/ui/workspaces/db-console/src/redux/alerts.spec.ts +++ b/pkg/ui/workspaces/db-console/src/redux/alerts.spec.ts @@ -31,6 +31,8 @@ import { disconnectedDismissedLocalSetting, emailSubscriptionAlertLocalSetting, emailSubscriptionAlertSelector, + clusterPreserveDowngradeOptionDismissedSetting, + clusterPreserveDowngradeOptionOvertimeSelector, } from "./alerts"; import { versionsSelector } from "src/redux/nodes"; import { @@ -45,7 +47,9 @@ import { nodesReducerObj, clusterReducerObj, healthReducerObj, + settingsReducerObj, } from "./apiReducers"; +import Long from "long"; import MembershipStatus = cockroach.kv.kvserver.liveness.livenesspb.MembershipStatus; const sandbox = sinon.createSandbox(); @@ -478,6 +482,57 @@ describe("alerts", function () { assert.isFalse(openState); }); }); + + describe("cluster.preserve_downgrade_option overtime alert", () => { + it("initialized with default false state", () => { + const settingState = + clusterPreserveDowngradeOptionDismissedSetting.selector(state()); + assert.isFalse(settingState); + }); + it("returns an alert if cluster.preserve_downgrad_option is lastUpdated >48 hours ago", () => { + dispatch( + settingsReducerObj.receiveData( + new protos.cockroach.server.serverpb.SettingsResponse({ + key_values: { + "cluster.preserve_downgrade_option": { + last_updated: { + seconds: Long.fromInt(165000000), + nanos: 165000000, + }, + value: "22.1", + }, + }, + }), + ), + ); + const alert = clusterPreserveDowngradeOptionOvertimeSelector(state()); + assert.isNotEmpty(alert); + }); + it("does not display alert once dismissed", async () => { + dispatch( + settingsReducerObj.receiveData( + new protos.cockroach.server.serverpb.SettingsResponse({ + key_values: { + "cluster.preserve_downgrade_option": { + last_updated: { + seconds: Long.fromInt(165000000), + nanos: 165000000, + }, + value: "22.1", + }, + }, + }), + ), + ); + + // dismiss alert + const alert = clusterPreserveDowngradeOptionOvertimeSelector(state()); + await alert.dismiss(dispatch, state); + const openState = + clusterPreserveDowngradeOptionDismissedSetting.selector(state()); + assert.isTrue(openState); + }); + }); }); describe("data sync listener", function () { @@ -592,7 +647,11 @@ describe("alerts", function () { new protos.cockroach.server.serverpb.ClusterResponse({}), ), ); - + dispatch( + settingsReducerObj.receiveData( + new protos.cockroach.server.serverpb.SettingsResponse({}), + ), + ); const expectedState = state(); sync(); assert.deepEqual(state(), expectedState); diff --git a/pkg/ui/workspaces/db-console/src/redux/alerts.ts b/pkg/ui/workspaces/db-console/src/redux/alerts.ts index 2f239b57ab2b..0e0238ac74c3 100644 --- a/pkg/ui/workspaces/db-console/src/redux/alerts.ts +++ b/pkg/ui/workspaces/db-console/src/redux/alerts.ts @@ -34,6 +34,7 @@ import { refreshNodes, refreshVersion, refreshHealth, + refreshSettings, } from "./apiReducers"; import { singleVersionSelector, @@ -42,6 +43,8 @@ import { import { AdminUIState, AppDispatch } from "./state"; import * as docsURL from "src/util/docs"; import { getDataFromServer } from "../util/dataFromServer"; +import { selectClusterSettings } from "./clusterSettings"; +import { longToInt } from "src/util/fixLong"; export enum AlertLevel { NOTIFICATION, @@ -511,6 +514,52 @@ export const terminateQueryAlertSelector = createSelector( }, ); +/** + * Notification for when the cluster.preserve_downgrade_option has been set for + * too long of a duration (48hrs) as part of a version upgrade. + */ +export const clusterPreserveDowngradeOptionDismissedSetting = new LocalSetting( + "cluster_preserve_downgrade_option_dismissed", + localSettingsSelector, + false, +); + +export const clusterPreserveDowngradeOptionOvertimeSelector = createSelector( + selectClusterSettings, + clusterPreserveDowngradeOptionDismissedSetting.selector, + (settings, notificationDismissed): Alert => { + if (notificationDismissed || !settings) { + return undefined; + } + const clusterPreserveDowngradeOption = + settings["cluster.preserve_downgrade_option"]; + const value = clusterPreserveDowngradeOption?.value; + const lastUpdated = clusterPreserveDowngradeOption?.last_updated; + if (!value || !lastUpdated) { + return undefined; + } + const lastUpdatedTime = moment.unix(longToInt(lastUpdated.seconds)); + const diff = moment.duration(moment().diff(lastUpdatedTime)).asHours(); + const maximumSetTime = 48; + if (diff < maximumSetTime) { + return undefined; + } + return { + level: AlertLevel.WARNING, + title: `Cluster setting cluster.preserve_downgrade_option has been set for greater than ${maximumSetTime} hours`, + text: `You can see a list of all nodes and their versions below. + Once all cluster nodes have been upgraded, and you have validated the stability and performance of + your workload on the new version, you must reset the cluster.preserve_downgrade_option cluster + setting with the following command: + RESET CLUSTER SETTING cluster.preserve_downgrade_option;`, + dismiss: (dispatch: AppDispatch) => { + dispatch(clusterPreserveDowngradeOptionDismissedSetting.set(true)); + return Promise.resolve(); + }, + }; + }, +); + /** * Selector which returns an array of all active alerts which should be * displayed in the overview list page, these should be non-critical alerts. @@ -518,6 +567,7 @@ export const terminateQueryAlertSelector = createSelector( export const overviewListAlertsSelector = createSelector( staggeredVersionWarningSelector, + clusterPreserveDowngradeOptionOvertimeSelector, (...alerts: Alert[]): Alert[] => { return _.without(alerts, null, undefined); }, @@ -620,16 +670,22 @@ export function alertDataSync(store: Store) { // Load Cluster ID once at startup. const cluster = state.cachedData.cluster; - if (cluster && !cluster.data && !cluster.inFlight) { + if (cluster && !cluster.data && !cluster.inFlight && !cluster.valid) { dispatch(refreshCluster()); } // Load Nodes initially if it has not yet been loaded. const nodes = state.cachedData.nodes; - if (nodes && !nodes.data && !nodes.inFlight) { + if (nodes && !nodes.data && !nodes.inFlight && !nodes.valid) { dispatch(refreshNodes()); } + // Load settings if not loaded + const settings = state.cachedData.settings; + if (settings && !settings.data && !settings.inFlight && !settings.valid) { + dispatch(refreshSettings()); + } + // Load potential new versions from CockroachDB cluster. This is the // complicating factor of this function, since the call requires the cluster // ID and node statuses being loaded first and thus cannot simply run at From 192f40c51fc98b9f2126122ad5cf0d5469ad703a Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Mon, 27 Jun 2022 09:17:45 +0100 Subject: [PATCH 2/2] jobs: add tracing spans to update and load job Here we start child tracing spans during update and load job operations. These seems like the two most interesting operations from the pespective of resumers. Release note: None --- pkg/jobs/jobs.go | 4 ++++ pkg/jobs/update.go | 6 +++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go index c10c49d17fa4..58399e7b3f93 100644 --- a/pkg/jobs/jobs.go +++ b/pkg/jobs/jobs.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" "github.com/gogo/protobuf/jsonpb" @@ -798,6 +799,9 @@ func HasJobNotFoundError(err error) bool { } func (j *Job) load(ctx context.Context, txn *kv.Txn) error { + ctx, sp := tracing.ChildSpan(ctx, "load-job") + defer sp.Finish() + var payload *jobspb.Payload var progress *jobspb.Progress var createdBy *CreatedByInfo diff --git a/pkg/jobs/update.go b/pkg/jobs/update.go index 4083276021c0..28122d16129c 100644 --- a/pkg/jobs/update.go +++ b/pkg/jobs/update.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" ) @@ -140,6 +141,9 @@ func (j *Job) Update(ctx context.Context, txn *kv.Txn, updateFn UpdateFn) error } func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateFn UpdateFn) error { + ctx, sp := tracing.ChildSpan(ctx, "update-job") + defer sp.Finish() + var payload *jobspb.Payload var progress *jobspb.Progress var status Status @@ -150,7 +154,7 @@ func (j *Job) update(ctx context.Context, txn *kv.Txn, useReadLock bool, updateF var err error var row tree.Datums row, err = j.registry.ex.QueryRowEx( - ctx, "log-job", txn, + ctx, "select-job", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, getSelectStmtForJobUpdate(j.session != nil, useReadLock), j.ID(), )