Skip to content

Commit

Permalink
Merge #82633 #83401
Browse files Browse the repository at this point in the history
82633: [CRDB-14199] ui, sql: add cluster.preserve_downgrade_option to prometheus and alerts r=Santamaura a=Santamaura

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.

on `_status/vars`
![Screen Shot 2022-06-09 at 1 01 48 PM](https://user-images.githubusercontent.com/17861665/172903600-b31c97dc-de66-4963-b48d-20a0b036e9df.png)

on dashboard
![Screen Shot 2022-06-08 at 2 06 43 PM](https://user-images.githubusercontent.com/17861665/172903656-59e7354d-6863-4668-a42b-f7806d56f60f.png)



83401: jobs: add tracing spans to update and load job r=miretskiy a=stevendanna

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

Co-authored-by: Santamaura <[email protected]>
Co-authored-by: Steven Danna <[email protected]>
  • Loading branch information
3 people committed Jun 27, 2022
3 parents 0598a36 + 2f96eeb + 192f40c commit dc5a394
Show file tree
Hide file tree
Showing 8 changed files with 181 additions and 4 deletions.
2 changes: 2 additions & 0 deletions pkg/clusterversion/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
40 changes: 40 additions & 0 deletions pkg/clusterversion/setting.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -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,
}
}
4 changes: 4 additions & 0 deletions pkg/jobs/jobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down
6 changes: 5 additions & 1 deletion pkg/jobs/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)

Expand Down Expand Up @@ -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
Expand All @@ -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(),
)
Expand Down
5 changes: 5 additions & 0 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -306,6 +306,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)
Expand Down
7 changes: 7 additions & 0 deletions pkg/ts/catalog/chart_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{
Expand Down
61 changes: 60 additions & 1 deletion pkg/ui/workspaces/db-console/src/redux/alerts.spec.ts
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ import {
disconnectedDismissedLocalSetting,
emailSubscriptionAlertLocalSetting,
emailSubscriptionAlertSelector,
clusterPreserveDowngradeOptionDismissedSetting,
clusterPreserveDowngradeOptionOvertimeSelector,
} from "./alerts";
import { versionsSelector } from "src/redux/nodes";
import {
Expand All @@ -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();
Expand Down Expand Up @@ -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 () {
Expand Down Expand Up @@ -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);
Expand Down
60 changes: 58 additions & 2 deletions pkg/ui/workspaces/db-console/src/redux/alerts.ts
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import {
refreshNodes,
refreshVersion,
refreshHealth,
refreshSettings,
} from "./apiReducers";
import {
singleVersionSelector,
Expand All @@ -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,
Expand Down Expand Up @@ -512,13 +515,60 @@ 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.
*/

export const overviewListAlertsSelector = createSelector(
staggeredVersionWarningSelector,
clusterPreserveDowngradeOptionOvertimeSelector,
(...alerts: Alert[]): Alert[] => {
return _.without(alerts, null, undefined);
},
Expand Down Expand Up @@ -621,16 +671,22 @@ export function alertDataSync(store: Store<AdminUIState>) {

// 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
Expand Down

0 comments on commit dc5a394

Please sign in to comment.