From d74e8a1c239580585f709d48d26da7abaf59ebb4 Mon Sep 17 00:00:00 2001 From: Tobias Schottdorf Date: Tue, 8 May 2018 22:54:28 -0400 Subject: [PATCH] server: add health checks and distress gossip While generating NodeStatus summaries, check for metrics that indicate a severe store, node, or cluster-level problem. Metrics can either be counters or gauges, and for the former we have to keep state so that we can notify only when the counter increments (not when it's nonzero). Flagged metrics are gossiped under a newly introduced key. These infos are in turn picked up by the newly introduced internal `gossip_alerts` table. In effect, operators can monitor the `crdb_internal.gossip_alerts` table on any node (though they'll want to do it on all nodes if there are network connectivity issues). Similarly, it'll be straightforward to plumb these warnings into the UI, though to the best of my knowledge the UI can't just query `crdb_internal`, and we may expose them in another more suitable location (this is trivial since Gossip is available pretty much everywhere). For starters, we only check the metrics for underreplicated and unavailable ranges as well as liveness errors, but there is no limitation on how elaborate these health checks can become. In fact, they aren't limited to sourcing solely from `NodeStatus`, and in light of #25316 we should consider alerting when nodes can't (bidirectionally) communicate, so that operators can easily diagnose DNS or firewall issues. NB: I had originally envisioned polling the `node_metrics` table because that would have allowed us to write the health checks in SQL. I had this code but ultimately deleted it as it seemed too roundabout and less extensible. Release note: None --- pkg/gossip/keys.go | 14 +- pkg/server/node.go | 38 +- pkg/server/node_test.go | 4 +- pkg/server/server.go | 2 +- pkg/server/server_test.go | 41 ++ pkg/server/status/health_check.go | 166 +++++++ pkg/server/status/health_check_test.go | 154 ++++++ pkg/server/status/recorder.go | 59 ++- pkg/server/status/recorder_test.go | 4 +- pkg/server/status/status.pb.go | 448 ++++++++++++++++-- pkg/server/status/status.proto | 25 + pkg/server/testserver.go | 2 +- pkg/server/updates.go | 2 +- pkg/server/updates_test.go | 2 +- pkg/sql/crdb_internal.go | 74 ++- .../testdata/logic_test/crdb_internal | 4 + pkg/sql/logictest/testdata/logic_test/explain | 6 +- .../logictest/testdata/logic_test/grant_table | 124 ++--- .../testdata/logic_test/information_schema | 55 ++- pkg/storage/metrics.go | 6 +- pkg/storage/store.go | 4 +- 21 files changed, 1050 insertions(+), 184 deletions(-) create mode 100644 pkg/server/status/health_check.go create mode 100644 pkg/server/status/health_check_test.go diff --git a/pkg/gossip/keys.go b/pkg/gossip/keys.go index de7b30f0e4a6..ee9c96cf20ce 100644 --- a/pkg/gossip/keys.go +++ b/pkg/gossip/keys.go @@ -53,6 +53,10 @@ const ( // string address of the node. E.g. node:1 => 127.0.0.1:24001 KeyNodeIDPrefix = "node" + // KeyHealthAlertPrefix is the key prefix for gossiping health alerts. The + // value is a proto of type HealthCheckResult. + KeyNodeHealthAlertPrefix = "health-alert" + // KeyNodeLivenessPrefix is the key prefix for gossiping node liveness info. KeyNodeLivenessPrefix = "liveness" @@ -112,8 +116,8 @@ func IsNodeIDKey(key string) bool { return strings.HasPrefix(key, KeyNodeIDPrefix+separator) } -// NodeIDFromKey attempts to extract a NodeID from the provided key. -// The key should have been constructed by MakeNodeIDKey. +// NodeIDFromKey attempts to extract a NodeID from the provided key. The key +// should have been constructed by MakeNodeIDKey or MakeNodeHealthAlertKey. // Returns an error if the key is not of the correct type or is not parsable. func NodeIDFromKey(key string) (roachpb.NodeID, error) { trimmedKey, err := removePrefixFromKey(key, KeyNodeIDPrefix) @@ -127,6 +131,12 @@ func NodeIDFromKey(key string) (roachpb.NodeID, error) { return roachpb.NodeID(nodeID), nil } +// MakeNodeHealthAlertKey returns the gossip key under which the given node can +// gossip health alerts. +func MakeNodeHealthAlertKey(nodeID roachpb.NodeID) string { + return MakeKey(KeyNodeHealthAlertPrefix, strconv.Itoa(int(nodeID))) +} + // MakeNodeLivenessKey returns the gossip key for node liveness info. func MakeNodeLivenessKey(nodeID roachpb.NodeID) string { return MakeKey(KeyNodeLivenessPrefix, nodeID.String()) diff --git a/pkg/server/node.go b/pkg/server/node.go index f9750b89970e..4dfaefe8cc43 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -634,7 +634,7 @@ func (n *Node) bootstrapStores( } // write a new status summary after all stores have been bootstrapped; this // helps the UI remain responsive when new nodes are added. - if err := n.writeSummaries(ctx); err != nil { + if err := n.writeNodeStatus(ctx, 0 /* alertTTL */); err != nil { log.Warningf(ctx, "error writing node summary after store bootstrap: %s", err) } } @@ -758,15 +758,15 @@ func (n *Node) computePeriodicMetrics(ctx context.Context, tick int) error { }) } -// startWriteSummaries begins periodically persisting status summaries for the +// startWriteNodeStatus begins periodically persisting status summaries for the // node and its stores. -func (n *Node) startWriteSummaries(frequency time.Duration) { +func (n *Node) startWriteNodeStatus(frequency time.Duration) { ctx := log.WithLogTag(n.AnnotateCtx(context.Background()), "summaries", nil) // Immediately record summaries once on server startup. n.stopper.RunWorker(ctx, func(ctx context.Context) { // Write a status summary immediately; this helps the UI remain // responsive when new nodes are added. - if err := n.writeSummaries(ctx); err != nil { + if err := n.writeNodeStatus(ctx, 0 /* alertTTL */); err != nil { log.Warningf(ctx, "error recording initial status summaries: %s", err) } ticker := time.NewTicker(frequency) @@ -774,7 +774,11 @@ func (n *Node) startWriteSummaries(frequency time.Duration) { for { select { case <-ticker.C: - if err := n.writeSummaries(ctx); err != nil { + // Use an alertTTL of twice the ticker frequency. This makes sure that + // alerts don't disappear and reappear spuriously while at the same + // time ensuring that an alert doesn't linger for too long after having + // resolved. + if err := n.writeNodeStatus(ctx, 2*frequency); err != nil { log.Warningf(ctx, "error recording status summaries: %s", err) } case <-n.stopper.ShouldStop(): @@ -784,12 +788,30 @@ func (n *Node) startWriteSummaries(frequency time.Duration) { }) } -// writeSummaries retrieves status summaries from the supplied +// writeNodeStatus retrieves status summaries from the supplied // NodeStatusRecorder and persists them to the cockroach data store. -func (n *Node) writeSummaries(ctx context.Context) error { +func (n *Node) writeNodeStatus(ctx context.Context, alertTTL time.Duration) error { var err error if runErr := n.stopper.RunTask(ctx, "node.Node: writing summary", func(ctx context.Context) { - err = n.recorder.WriteStatusSummary(ctx, n.storeCfg.DB) + nodeStatus := n.recorder.GenerateNodeStatus(ctx) + if nodeStatus == nil { + return + } + + if result := n.recorder.CheckHealth(ctx, *nodeStatus); len(result.Alerts) != 0 { + log.Warningf(ctx, "health alerts detected: %+v", result) + if err := n.storeCfg.Gossip.AddInfoProto( + gossip.MakeNodeHealthAlertKey(n.Descriptor.NodeID), &result, alertTTL, + ); err != nil { + log.Warningf(ctx, "unable to gossip health alerts: %+v", result) + } + + // TODO(tschottdorf): add a metric that we increment every time there are + // alerts. This can help understand how long the cluster has been in that + // state (since it'll be incremented every ~10s). + } + + err = n.recorder.WriteNodeStatus(ctx, n.storeCfg.DB, *nodeStatus) }); runErr != nil { err = runErr } diff --git a/pkg/server/node_test.go b/pkg/server/node_test.go index f3be1194023f..7231f799624b 100644 --- a/pkg/server/node_test.go +++ b/pkg/server/node_test.go @@ -524,9 +524,9 @@ func compareNodeStatus( return nodeStatus } -// TestStatusSummaries verifies that status summaries are written correctly for +// TestNodeStatusWritten verifies that status summaries are written correctly for // both the Node and stores within the node. -func TestStatusSummaries(t *testing.T) { +func TestNodeStatusWritten(t *testing.T) { defer leaktest.AfterTest(t)() // ======================================== diff --git a/pkg/server/server.go b/pkg/server/server.go index 6d3834f01905..2bf29caa47bb 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -1440,7 +1440,7 @@ If problems persist, please see ` + base.DocsURL("cluster-setup-troubleshooting. ) // Begin recording status summaries. - s.node.startWriteSummaries(DefaultMetricsSampleInterval) + s.node.startWriteNodeStatus(DefaultMetricsSampleInterval) // Create and start the schema change manager only after a NodeID // has been assigned. diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index cb232d1b5ffc..8b08f9b08dbd 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -36,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/server/status" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/engine" @@ -70,6 +71,46 @@ func TestSelfBootstrap(t *testing.T) { } } +// TestHealthCheck runs a basic sanity check on the health checker. +func TestHealthCheck(t *testing.T) { + defer leaktest.AfterTest(t)() + s, err := serverutils.StartServerRaw(base.TestServerArgs{}) + if err != nil { + t.Fatal(err) + } + defer s.Stopper().Stop(context.TODO()) + + ctx := context.Background() + + recorder := s.(*TestServer).Server.recorder + + { + summary := *recorder.GenerateNodeStatus(ctx) + result := recorder.CheckHealth(ctx, summary) + if len(result.Alerts) != 0 { + t.Fatal(result) + } + } + + store, err := s.(*TestServer).Server.node.stores.GetStore(1) + if err != nil { + t.Fatal(err) + } + + store.Metrics().UnavailableRangeCount.Inc(100) + + { + summary := *recorder.GenerateNodeStatus(ctx) + result := recorder.CheckHealth(ctx, summary) + expAlerts := []status.HealthAlert{ + {StoreID: 1, Category: status.HealthAlert_METRICS, Description: "ranges.unavailable", Value: 100.0}, + } + if !reflect.DeepEqual(expAlerts, result.Alerts) { + t.Fatalf("expected %+v, got %+v", expAlerts, result.Alerts) + } + } +} + // TestServerStartClock tests that a server's clock is not pushed out of thin // air. This used to happen - the simple act of starting was causing a server's // clock to be pushed because we were introducing bogus future timestamps into diff --git a/pkg/server/status/health_check.go b/pkg/server/status/health_check.go new file mode 100644 index 000000000000..fa26bde91c7e --- /dev/null +++ b/pkg/server/status/health_check.go @@ -0,0 +1,166 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package status + +import ( + "context" + + "time" + + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +type threshold struct { + gauge bool + min int64 +} + +var ( + counterZero = threshold{} + gaugeZero = threshold{gauge: true} +) + +// TODO(tschottdorf): I think we should just export the metric metadata from +// their respective packages and reference them here, instead of the +// duplication. It also seems useful to specify the metric type in the metadata +// so that we don't have to "guess" whether it's a gauge or counter. However +// there's some massaging for latency histograms that happens in NodeStatus, +// so the logic likely has to be moved up a bit. A thread not worth pulling on +// at the moment, I suppose. +// +// TODO(tschottdorf): there are some metrics that could be used in alerts but +// need special treatment. For example, we want to alert when compactions are +// queued but not processed over long periods of time, or when queues have a +// large backlog but show no sign of processing times. +var trackedMetrics = map[string]threshold{ + // Gauges. + "ranges.unavailable": gaugeZero, + "ranges.underreplicated": gaugeZero, + "requests.backpressure.split": gaugeZero, + "requests.slow.commandqueue": gaugeZero, + "requests.slow.distsender": gaugeZero, + "requests.slow.lease": gaugeZero, + "requests.slow.raft": gaugeZero, + "sys.goroutines": {gauge: true, min: 5000}, + + // Latencies (which are really histograms, but we get to see a fixed number + // of percentiles as gauges) + "raft.process.logcommit.latency-90": {gauge: true, min: int64(500 * time.Millisecond)}, + "round-trip-latency-p90": {gauge: true, min: int64(time.Second)}, + + // Counters. + + "liveness.heartbeatfailures": counterZero, + "timeseries.write.errors": counterZero, + + // Queue processing errors. This might be too aggressive. For example, if the + // replicate queue is waiting for a split, does that generate an error? If so, + // is that worth alerting about? We might need severities here at some point + // or some other way to guard against "blips". + "compactor.compactions.failure": counterZero, + "queue.replicagc.process.failure": counterZero, + "queue.raftlog.process.failure": counterZero, + "queue.gc.process.failure": counterZero, + "queue.split.process.failure": counterZero, + "queue.replicate.process.failure": counterZero, + "queue.raftsnapshot.process.failure": counterZero, + "queue.tsmaintenance.process.failure": counterZero, + "queue.consistency.process.failure": counterZero, +} + +type metricsMap map[roachpb.StoreID]map[string]float64 + +// update takes a populated metrics map and extracts the tracked metrics. Gauges +// are returned verbatim, while for counters the diff between the last seen +// value is returned. Only nonzero values are reported and the seen (non-relative) +// values are persisted for the next call. +func (d metricsMap) update(tracked map[string]threshold, m metricsMap) metricsMap { + out := metricsMap{} + for storeID := range m { + for name, threshold := range tracked { + val, ok := m[storeID][name] + if !ok { + continue + } + + if !threshold.gauge { + prevVal, havePrev := d[storeID][name] + if d[storeID] == nil { + d[storeID] = map[string]float64{} + } + d[storeID][name] = val + if havePrev { + val -= prevVal + } else { + // Can't report the first time around if we don't know the previous + // value of the counter. + val = 0 + } + } + + if val > float64(threshold.min) { + if out[storeID] == nil { + out[storeID] = map[string]float64{} + } + out[storeID][name] = val + } + } + } + return out +} + +// A HealthChecker inspects the node metrics and optionally a NodeStatus for +// anomalous conditions that the operator should be alerted to. +type HealthChecker struct { + state metricsMap + tracked map[string]threshold +} + +// NewHealthChecker creates a new health checker that emits alerts whenever the +// given metrics are nonzero. Setting the boolean map value indicates a gauge +// (in which case it is reported whenever it's nonzero); otherwise the metric is +// treated as a counter and reports whenever it is incremented between +// consecutive calls of `CheckHealth`. +func NewHealthChecker(trackedMetrics map[string]threshold) *HealthChecker { + return &HealthChecker{state: metricsMap{}, tracked: trackedMetrics} +} + +// CheckHealth performs a (cheap) health check. It is not thread safe. +func (h *HealthChecker) CheckHealth(ctx context.Context, nodeStatus NodeStatus) HealthCheckResult { + // Gauges that trigger alerts when nonzero. + var alerts []HealthAlert + + m := map[roachpb.StoreID]map[string]float64{ + 0: nodeStatus.Metrics, + } + for _, storeStatus := range nodeStatus.StoreStatuses { + m[storeStatus.Desc.StoreID] = storeStatus.Metrics + } + + diffs := h.state.update(h.tracked, m) + + for storeID, storeDiff := range diffs { + for name, value := range storeDiff { + alerts = append(alerts, HealthAlert{ + StoreID: storeID, + Category: HealthAlert_METRICS, + Description: name, + Value: value, + }) + } + } + + return HealthCheckResult{Alerts: alerts} +} diff --git a/pkg/server/status/health_check_test.go b/pkg/server/status/health_check_test.go new file mode 100644 index 000000000000..f02af6c5f48c --- /dev/null +++ b/pkg/server/status/health_check_test.go @@ -0,0 +1,154 @@ +// Copyright 2018 The Cockroach Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or +// implied. See the License for the specific language governing +// permissions and limitations under the License. + +package status + +import ( + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/kr/pretty" +) + +func TestHealthCheckMetricsMap(t *testing.T) { + defer leaktest.AfterTest(t)() + + m := metricsMap{} + + tracked := map[string]threshold{ + "gauge0": gaugeZero, + "gauge1": gaugeZero, + "gauge2": {gauge: true, min: 100}, + "counter0": counterZero, + "counter1": counterZero, + "counter2": {min: 100}, + } + + check := func(act, exp metricsMap) { + t.Helper() + if diff := pretty.Diff(act, exp); len(diff) != 0 { + t.Fatalf("diff(act,exp) = %s\n\nact=%+v\nexp=%+v", strings.Join(diff, "\n"), act, exp) + } + } + + // A gauge and a counter show up. + check(m.update(tracked, metricsMap{ + 0: { + "gauge0": 1, + "counter0": 12, + }, + 1: { + "gauge0": 10, + "counter2": 0, + }, + }), metricsMap{ + 0: {"gauge0": 1}, + 1: {"gauge0": 10}, + }) + + check(m, metricsMap{ + 0: { + "counter0": 12, + }, + 1: { + "counter2": 0, + }, + }) + + // A counter increments for the second time, and another one shows up for the + // first time. The thresholded counter moves, but stays below threshold. + check(m.update(tracked, metricsMap{ + 0: { + "counter0": 14, + "counter1": 5, + }, + 1: { + "counter2": 100, // barely misses threshold + }, + }), metricsMap{ + 0: {"counter0": 2}, + }) + + check(m, metricsMap{ + 0: { + "counter0": 14, + "counter1": 5, + }, + 1: { + "counter2": 100, + }, + }) + + // A gauge shows up for the second time. A counter we've seen before increments, + // but on a different store (so it's really a first time still). The thresholded + // counter jumps by the threshold value (note that counter thresholds aren't + // really that useful, except as a very poor man's rate limiter). + check(m.update(tracked, metricsMap{ + 1: { + "gauge0": 12, + "counter1": 9, + "counter2": 201, + }, + }), metricsMap{ + 1: { + "gauge0": 12, + "counter2": 101, + }, + }) + + check(m, metricsMap{ + 0: { + "counter0": 14, + "counter1": 5, + }, + 1: { + "counter1": 9, + "counter2": 201, + }, + }) + + // Both metrics we've seen before change. One increments (expected) and one + // decrements (we never do that in practice). + check(m.update(tracked, metricsMap{ + 0: { + "counter0": 3, + "counter1": 10, + }, + 1: { + "counter1": 4, + }, + }), metricsMap{ + 0: { + "counter1": 5, + }, + }) + + finalMap := metricsMap{ + 0: { + "counter0": 3, + "counter1": 10, + }, + 1: { + "counter1": 4, + "counter2": 201, + }, + } + check(m, finalMap) + + // Nothing changes, except something we don't track. + check(m.update(tracked, metricsMap{1: {"banana": 100}}), metricsMap{}) + check(m.update(tracked, metricsMap{1: {"banana": 300}}), metricsMap{}) + check(m, finalMap) +} diff --git a/pkg/server/status/recorder.go b/pkg/server/status/recorder.go index 9d40443effca..d1f6d7daf9ab 100644 --- a/pkg/server/status/recorder.go +++ b/pkg/server/status/recorder.go @@ -89,6 +89,7 @@ type storeMetrics interface { // store hosted by the node. There are slight differences in the way these are // recorded, and they are thus kept separate. type MetricsRecorder struct { + *HealthChecker gossip *gossip.Gossip nodeLiveness *storage.NodeLiveness rpcContext *rpc.Context @@ -129,7 +130,7 @@ type MetricsRecorder struct { // prometheus text format. prometheusExporter metric.PrometheusExporter } - // WriteStatusSummary is a potentially long-running method (with a network + // WriteNodeStatus is a potentially long-running method (with a network // round-trip) that requires a mutex to be safe for concurrent usage. We // therefore give it its own mutex to avoid blocking other methods. writeSummaryMu syncutil.Mutex @@ -145,10 +146,11 @@ func NewMetricsRecorder( settings *cluster.Settings, ) *MetricsRecorder { mr := &MetricsRecorder{ - nodeLiveness: nodeLiveness, - rpcContext: rpcContext, - gossip: gossip, - settings: settings, + HealthChecker: NewHealthChecker(trackedMetrics), + nodeLiveness: nodeLiveness, + rpcContext: rpcContext, + gossip: gossip, + settings: settings, } mr.mu.storeRegistries = make(map[roachpb.StoreID]*metric.Registry) mr.mu.stores = make(map[roachpb.StoreID]storeMetrics) @@ -344,10 +346,10 @@ func (mr *MetricsRecorder) getNetworkActivity( return activity } -// GetStatusSummary returns a status summary message for the node. The summary +// GenerateNodeStatus returns a status summary message for the node. The summary // includes the recent values of metrics for both the node and all of its -// component stores. -func (mr *MetricsRecorder) GetStatusSummary(ctx context.Context) *NodeStatus { +// component stores. When the node isn't initialized yet, nil is returned. +func (mr *MetricsRecorder) GenerateNodeStatus(ctx context.Context) *NodeStatus { activity := mr.getNetworkActivity(ctx) mr.mu.RLock() @@ -425,31 +427,28 @@ func (mr *MetricsRecorder) GetStatusSummary(ctx context.Context) *NodeStatus { return nodeStat } -// WriteStatusSummary generates a summary and immediately writes it to the given -// client. -func (mr *MetricsRecorder) WriteStatusSummary(ctx context.Context, db *client.DB) error { +// WriteNodeStatus writes the supplied summary to the given client. +func (mr *MetricsRecorder) WriteNodeStatus( + ctx context.Context, db *client.DB, nodeStatus NodeStatus, +) error { mr.writeSummaryMu.Lock() defer mr.writeSummaryMu.Unlock() - - nodeStatus := mr.GetStatusSummary(ctx) - if nodeStatus != nil { - key := keys.NodeStatusKey(nodeStatus.Desc.NodeID) - // We use PutInline to store only a single version of the node status. - // There's not much point in keeping the historical versions as we keep - // all of the constituent data as timeseries. Further, due to the size - // of the build info in the node status, writing one of these every 10s - // will generate more versions than will easily fit into a range over - // the course of a day. - if err := db.PutInline(ctx, key, nodeStatus); err != nil { - return err - } - if log.V(2) { - statusJSON, err := json.Marshal(nodeStatus) - if err != nil { - log.Errorf(ctx, "error marshaling nodeStatus to json: %s", err) - } - log.Infof(ctx, "node %d status: %s", nodeStatus.Desc.NodeID, statusJSON) + key := keys.NodeStatusKey(nodeStatus.Desc.NodeID) + // We use PutInline to store only a single version of the node status. + // There's not much point in keeping the historical versions as we keep + // all of the constituent data as timeseries. Further, due to the size + // of the build info in the node status, writing one of these every 10s + // will generate more versions than will easily fit into a range over + // the course of a day. + if err := db.PutInline(ctx, key, &nodeStatus); err != nil { + return err + } + if log.V(2) { + statusJSON, err := json.Marshal(&nodeStatus) + if err != nil { + log.Errorf(ctx, "error marshaling nodeStatus to json: %s", err) } + log.Infof(ctx, "node %d status: %s", nodeStatus.Desc.NodeID, statusJSON) } return nil } diff --git a/pkg/server/status/recorder_test.go b/pkg/server/status/recorder_test.go index 6a15d3a0d25a..900412ccfd55 100644 --- a/pkg/server/status/recorder_test.go +++ b/pkg/server/status/recorder_test.go @@ -349,7 +349,7 @@ func TestMetricsRecorder(t *testing.T) { t.Fatal(err) } - nodeSummary := recorder.GetStatusSummary(context.Background()) + nodeSummary := recorder.GenerateNodeStatus(context.Background()) if nodeSummary == nil { t.Fatalf("recorder did not return nodeSummary") } @@ -369,7 +369,7 @@ func TestMetricsRecorder(t *testing.T) { t.Errorf("recorder did not produce expected NodeSummary; diff:\n %s", pretty.Diff(e, a)) } - // Make sure that all methods other than GetStatusSummary can operate in + // Make sure that all methods other than GenerateNodeStatus can operate in // parallel with each other (i.e. even if recorder.mu is RLocked). recorder.mu.RLock() var wg sync.WaitGroup diff --git a/pkg/server/status/status.pb.go b/pkg/server/status/status.pb.go index 0ded6ea46bda..d0cbd365961b 100644 --- a/pkg/server/status/status.pb.go +++ b/pkg/server/status/status.pb.go @@ -10,6 +10,8 @@ It has these top-level messages: StoreStatus NodeStatus + HealthAlert + HealthCheckResult */ package status @@ -37,6 +39,27 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package +type HealthAlert_Category int32 + +const ( + HealthAlert_METRICS HealthAlert_Category = 0 + HealthAlert_NETWORK HealthAlert_Category = 1 +) + +var HealthAlert_Category_name = map[int32]string{ + 0: "METRICS", + 1: "NETWORK", +} +var HealthAlert_Category_value = map[string]int32{ + "METRICS": 0, + "NETWORK": 1, +} + +func (x HealthAlert_Category) String() string { + return proto.EnumName(HealthAlert_Category_name, int32(x)) +} +func (HealthAlert_Category) EnumDescriptor() ([]byte, []int) { return fileDescriptorStatus, []int{2, 0} } + // StoreStatus records the most recent values of metrics for a store. type StoreStatus struct { Desc cockroach_roachpb.StoreDescriptor `protobuf:"bytes,1,opt,name=desc" json:"desc"` @@ -87,10 +110,38 @@ func (*NodeStatus_NetworkActivity) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{1, 2} } +// A HealthAlert is an undesired condition detected by a server which should be +// exposed to the operators. +type HealthAlert struct { + // store_id is zero for alerts not specific to a store (i.e. apply at the node level). + StoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,1,opt,name=store_id,json=storeId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"store_id"` + Category HealthAlert_Category `protobuf:"varint,2,opt,name=category,proto3,enum=cockroach.server.status.HealthAlert_Category" json:"category,omitempty"` + Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` + Value float64 `protobuf:"fixed64,4,opt,name=value,proto3" json:"value,omitempty"` +} + +func (m *HealthAlert) Reset() { *m = HealthAlert{} } +func (m *HealthAlert) String() string { return proto.CompactTextString(m) } +func (*HealthAlert) ProtoMessage() {} +func (*HealthAlert) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{2} } + +// HealthCheckResult holds a number of HealthAlerts. +type HealthCheckResult struct { + Alerts []HealthAlert `protobuf:"bytes,1,rep,name=alerts" json:"alerts"` +} + +func (m *HealthCheckResult) Reset() { *m = HealthCheckResult{} } +func (m *HealthCheckResult) String() string { return proto.CompactTextString(m) } +func (*HealthCheckResult) ProtoMessage() {} +func (*HealthCheckResult) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{3} } + func init() { proto.RegisterType((*StoreStatus)(nil), "cockroach.server.status.StoreStatus") proto.RegisterType((*NodeStatus)(nil), "cockroach.server.status.NodeStatus") proto.RegisterType((*NodeStatus_NetworkActivity)(nil), "cockroach.server.status.NodeStatus.NetworkActivity") + proto.RegisterType((*HealthAlert)(nil), "cockroach.server.status.HealthAlert") + proto.RegisterType((*HealthCheckResult)(nil), "cockroach.server.status.HealthCheckResult") + proto.RegisterEnum("cockroach.server.status.HealthAlert_Category", HealthAlert_Category_name, HealthAlert_Category_value) } func (m *StoreStatus) Marshal() (dAtA []byte, err error) { size := m.Size() @@ -331,6 +382,76 @@ func (m *NodeStatus_NetworkActivity) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *HealthAlert) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *HealthAlert) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.StoreID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintStatus(dAtA, i, uint64(m.StoreID)) + } + if m.Category != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintStatus(dAtA, i, uint64(m.Category)) + } + if len(m.Description) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintStatus(dAtA, i, uint64(len(m.Description))) + i += copy(dAtA[i:], m.Description) + } + if m.Value != 0 { + dAtA[i] = 0x21 + i++ + binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value)))) + i += 8 + } + return i, nil +} + +func (m *HealthCheckResult) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *HealthCheckResult) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Alerts) > 0 { + for _, msg := range m.Alerts { + dAtA[i] = 0xa + i++ + i = encodeVarintStatus(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + func encodeVarintStatus(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -430,6 +551,37 @@ func (m *NodeStatus_NetworkActivity) Size() (n int) { return n } +func (m *HealthAlert) Size() (n int) { + var l int + _ = l + if m.StoreID != 0 { + n += 1 + sovStatus(uint64(m.StoreID)) + } + if m.Category != 0 { + n += 1 + sovStatus(uint64(m.Category)) + } + l = len(m.Description) + if l > 0 { + n += 1 + l + sovStatus(uint64(l)) + } + if m.Value != 0 { + n += 9 + } + return n +} + +func (m *HealthCheckResult) Size() (n int) { + var l int + _ = l + if len(m.Alerts) > 0 { + for _, e := range m.Alerts { + l = e.Size() + n += 1 + l + sovStatus(uint64(l)) + } + } + return n +} + func sovStatus(x uint64) (n int) { for { n++ @@ -1275,6 +1427,215 @@ func (m *NodeStatus_NetworkActivity) Unmarshal(dAtA []byte) error { } return nil } +func (m *HealthAlert) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: HealthAlert: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HealthAlert: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field StoreID", wireType) + } + m.StoreID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.StoreID |= (github_com_cockroachdb_cockroach_pkg_roachpb.StoreID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Category", wireType) + } + m.Category = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Category |= (HealthAlert_Category(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Description", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthStatus + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Description = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Value = float64(math.Float64frombits(v)) + default: + iNdEx = preIndex + skippy, err := skipStatus(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthStatus + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *HealthCheckResult) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: HealthCheckResult: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HealthCheckResult: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Alerts", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStatus + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStatus + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Alerts = append(m.Alerts, HealthAlert{}) + if err := m.Alerts[len(m.Alerts)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipStatus(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthStatus + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipStatus(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 @@ -1383,43 +1744,52 @@ var ( func init() { proto.RegisterFile("server/status/status.proto", fileDescriptorStatus) } var fileDescriptorStatus = []byte{ - // 596 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x54, 0x3f, 0x6f, 0xd3, 0x4e, - 0x18, 0xee, 0xd5, 0x69, 0x13, 0x5f, 0x7e, 0xed, 0xaf, 0x3a, 0x15, 0xb0, 0x2c, 0xe1, 0x9a, 0x88, - 0x21, 0x93, 0x0d, 0xc9, 0x82, 0xd2, 0x2e, 0x89, 0xca, 0x10, 0xfe, 0x54, 0xc2, 0xdd, 0x58, 0xa2, - 0x8b, 0x7d, 0x75, 0xad, 0x24, 0x3e, 0xeb, 0xee, 0x1c, 0x94, 0x95, 0x11, 0x09, 0x89, 0x8f, 0x95, - 0x91, 0x81, 0x81, 0x89, 0x3f, 0xe1, 0x3b, 0x30, 0xa3, 0x3b, 0x3b, 0xb1, 0x83, 0x12, 0x11, 0x09, - 0xa6, 0xbc, 0xf7, 0xbc, 0x7e, 0xde, 0xf7, 0x79, 0x9e, 0x3b, 0x05, 0x9a, 0x9c, 0xb0, 0x29, 0x61, - 0x2e, 0x17, 0x58, 0xa4, 0x3c, 0xff, 0x71, 0x12, 0x46, 0x05, 0x45, 0xf7, 0x7c, 0xea, 0x8f, 0x18, - 0xc5, 0xfe, 0xad, 0x93, 0x7d, 0xe5, 0x64, 0x6d, 0xf3, 0xae, 0x02, 0x93, 0xa1, 0x3b, 0x21, 0x02, - 0x07, 0x58, 0xe0, 0x8c, 0x60, 0x9e, 0x0c, 0xd3, 0x68, 0x1c, 0xb8, 0x51, 0x7c, 0x43, 0x73, 0xe4, - 0x34, 0xa4, 0x21, 0x55, 0xa5, 0x2b, 0xab, 0x0c, 0x6d, 0x7c, 0x02, 0xb0, 0x7e, 0x2d, 0x28, 0x23, - 0xd7, 0x6a, 0x1e, 0xba, 0x80, 0x95, 0x80, 0x70, 0xdf, 0x00, 0x36, 0x68, 0xd6, 0x5b, 0x0d, 0xa7, - 0xd8, 0x9b, 0x2f, 0x72, 0xd4, 0xd7, 0x97, 0x84, 0xfb, 0x2c, 0x4a, 0x04, 0x65, 0xbd, 0xca, 0xfc, - 0xcb, 0xd9, 0x9e, 0xa7, 0x58, 0xe8, 0x39, 0xac, 0x4e, 0x88, 0x60, 0x91, 0xcf, 0x8d, 0x7d, 0x5b, - 0x6b, 0xd6, 0x5b, 0x8f, 0x9d, 0x2d, 0xc2, 0x9d, 0xd2, 0x52, 0xe7, 0x65, 0xc6, 0x79, 0x1a, 0x0b, - 0x36, 0xf3, 0x96, 0x13, 0xcc, 0x0e, 0xfc, 0xaf, 0xdc, 0x40, 0x27, 0x50, 0x1b, 0x91, 0x99, 0x52, - 0xa6, 0x7b, 0xb2, 0x44, 0xa7, 0xf0, 0x60, 0x8a, 0xc7, 0x29, 0x31, 0xf6, 0x6d, 0xd0, 0x04, 0x5e, - 0x76, 0xe8, 0xec, 0x3f, 0x01, 0x8d, 0x9f, 0x55, 0x08, 0xaf, 0x68, 0xb0, 0x74, 0x75, 0xbe, 0xe6, - 0xea, 0xc1, 0x06, 0x57, 0xf2, 0xe3, 0x2d, 0xa6, 0x3a, 0x10, 0xaa, 0x30, 0x07, 0x32, 0x4c, 0xb5, - 0xaa, 0xde, 0xba, 0x53, 0x1a, 0xa1, 0x9a, 0x4e, 0x3f, 0xbe, 0xa1, 0x39, 0x4d, 0x57, 0x88, 0x04, - 0xd0, 0x7d, 0x08, 0xb9, 0xc0, 0x4c, 0x90, 0x60, 0x80, 0x85, 0xa1, 0xd9, 0xa0, 0xa9, 0x79, 0x7a, - 0x8e, 0x74, 0x85, 0x6c, 0xa7, 0x49, 0x80, 0xf3, 0x76, 0x25, 0x6b, 0xe7, 0x48, 0x57, 0xa0, 0x67, - 0x45, 0x9c, 0x07, 0x2a, 0xce, 0x47, 0x5b, 0xe3, 0x2c, 0xcc, 0x6e, 0x4e, 0x13, 0xbd, 0x82, 0xc7, - 0x5c, 0x46, 0x3e, 0xc8, 0x08, 0x84, 0x1b, 0x87, 0x6a, 0xe4, 0xc3, 0x5d, 0x6e, 0x28, 0x37, 0x76, - 0xc4, 0x0b, 0x88, 0x70, 0x84, 0x60, 0x05, 0xb3, 0x90, 0x1b, 0x55, 0x5b, 0x6b, 0xea, 0x9e, 0xaa, - 0xe5, 0x25, 0x91, 0x78, 0x6a, 0xd4, 0x14, 0x24, 0x4b, 0xf4, 0x1e, 0x40, 0x7d, 0x8c, 0x05, 0x89, - 0xfd, 0x88, 0x70, 0x43, 0x57, 0x4b, 0x5b, 0xbb, 0xf8, 0x78, 0xb1, 0x24, 0x29, 0x27, 0xbd, 0x73, - 0x29, 0xe1, 0xed, 0xd7, 0xb3, 0x76, 0x18, 0x89, 0xdb, 0x74, 0xe8, 0xf8, 0x74, 0xe2, 0xae, 0xa6, - 0x04, 0xc3, 0xa2, 0x76, 0x93, 0x51, 0xe8, 0x96, 0xef, 0xb5, 0x7f, 0xe9, 0x15, 0x0a, 0xd0, 0x3b, - 0x00, 0x6b, 0xd8, 0x17, 0xd1, 0x34, 0x12, 0x33, 0x03, 0xfe, 0xe1, 0x95, 0x96, 0xe4, 0x74, 0x73, - 0xce, 0x3f, 0x50, 0xb3, 0xda, 0xff, 0x37, 0x6f, 0xdc, 0xbc, 0x80, 0xc7, 0xeb, 0x11, 0x95, 0xd9, - 0x07, 0x1b, 0xd8, 0x5a, 0x99, 0xed, 0xc3, 0xff, 0xaf, 0x88, 0x78, 0x43, 0xd9, 0x68, 0x69, 0x0c, - 0x99, 0xb0, 0x16, 0xc5, 0x3e, 0x9d, 0x44, 0x71, 0xa8, 0x66, 0x68, 0xde, 0xea, 0x2c, 0x7b, 0x34, - 0x15, 0x21, 0x95, 0xbd, 0x6c, 0xd6, 0xea, 0x8c, 0x0c, 0x58, 0xcd, 0xe2, 0x9d, 0xe5, 0x2f, 0x7c, - 0x79, 0x34, 0x13, 0x78, 0xb4, 0x16, 0xdb, 0x06, 0x85, 0xfd, 0xb2, 0xc2, 0x7a, 0xab, 0xbd, 0xcb, - 0x55, 0xfc, 0x26, 0xbc, 0x64, 0xab, 0x67, 0xcf, 0xbf, 0x5b, 0x7b, 0xf3, 0x85, 0x05, 0x3e, 0x2e, - 0x2c, 0xf0, 0x79, 0x61, 0x81, 0x6f, 0x0b, 0x0b, 0x7c, 0xf8, 0x61, 0xed, 0xbd, 0x3e, 0xcc, 0xe6, - 0x0c, 0x0f, 0xd5, 0x1f, 0x5f, 0xfb, 0x57, 0x00, 0x00, 0x00, 0xff, 0xff, 0xc5, 0x22, 0x21, 0x49, - 0x6f, 0x05, 0x00, 0x00, + // 752 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x55, 0xcf, 0x6f, 0xda, 0x48, + 0x14, 0x66, 0x80, 0xf0, 0xe3, 0x79, 0x93, 0x65, 0x47, 0xd9, 0x5d, 0x0b, 0x69, 0x81, 0x45, 0x39, + 0x70, 0x59, 0xb3, 0x4b, 0xf6, 0x50, 0x25, 0xb9, 0x40, 0x12, 0xb5, 0x34, 0x4d, 0xaa, 0x3a, 0x91, + 0x22, 0xf5, 0x12, 0x0d, 0xf6, 0xc4, 0x58, 0x80, 0x07, 0x79, 0x06, 0x2a, 0xae, 0x3d, 0x56, 0xaa, + 0xd4, 0x73, 0xff, 0xa2, 0x1c, 0x7b, 0xe8, 0xa1, 0xea, 0x21, 0x6d, 0xe9, 0xff, 0xd0, 0x73, 0x35, + 0xe3, 0x01, 0x9c, 0x0a, 0x94, 0x54, 0xed, 0x89, 0x37, 0xef, 0xf9, 0x7b, 0xef, 0xfb, 0xbe, 0x37, + 0x36, 0x50, 0xe4, 0x34, 0x1c, 0xd3, 0xb0, 0xce, 0x05, 0x11, 0x23, 0xae, 0x7f, 0xac, 0x61, 0xc8, + 0x04, 0xc3, 0x7f, 0x3a, 0xcc, 0xe9, 0x85, 0x8c, 0x38, 0x5d, 0x2b, 0x7a, 0xca, 0x8a, 0xca, 0xc5, + 0x3f, 0x54, 0x72, 0xd8, 0xa9, 0x0f, 0xa8, 0x20, 0x2e, 0x11, 0x24, 0x02, 0x14, 0x0b, 0x9d, 0x91, + 0xdf, 0x77, 0xeb, 0x7e, 0x70, 0xc9, 0x74, 0x66, 0xd3, 0x63, 0x1e, 0x53, 0x61, 0x5d, 0x46, 0x51, + 0xb6, 0xfa, 0x16, 0x81, 0x71, 0x2a, 0x58, 0x48, 0x4f, 0x55, 0x3f, 0xbc, 0x07, 0x69, 0x97, 0x72, + 0xc7, 0x44, 0x15, 0x54, 0x33, 0x1a, 0x55, 0x6b, 0x31, 0x57, 0x0f, 0xb2, 0xd4, 0xd3, 0x07, 0x94, + 0x3b, 0xa1, 0x3f, 0x14, 0x2c, 0x6c, 0xa5, 0xaf, 0xae, 0xcb, 0x09, 0x5b, 0xa1, 0xf0, 0x11, 0x64, + 0x07, 0x54, 0x84, 0xbe, 0xc3, 0xcd, 0x64, 0x25, 0x55, 0x33, 0x1a, 0xff, 0x59, 0x2b, 0x88, 0x5b, + 0xb1, 0xa1, 0xd6, 0x71, 0x84, 0x39, 0x0c, 0x44, 0x38, 0xb1, 0x67, 0x1d, 0x8a, 0x3b, 0xf0, 0x4b, + 0xbc, 0x80, 0x0b, 0x90, 0xea, 0xd1, 0x89, 0x62, 0x96, 0xb7, 0x65, 0x88, 0x37, 0x61, 0x6d, 0x4c, + 0xfa, 0x23, 0x6a, 0x26, 0x2b, 0xa8, 0x86, 0xec, 0xe8, 0xb0, 0x93, 0xbc, 0x87, 0xaa, 0x5f, 0xb2, + 0x00, 0x27, 0xcc, 0x9d, 0xa9, 0xda, 0xbd, 0xa1, 0xea, 0xef, 0x25, 0xaa, 0xe4, 0xc3, 0x2b, 0x44, + 0xed, 0x00, 0x28, 0x33, 0x2f, 0xa4, 0x99, 0x6a, 0x94, 0xd1, 0xf8, 0x3d, 0xd6, 0x42, 0x15, 0xad, + 0x76, 0x70, 0xc9, 0x34, 0x2c, 0xaf, 0x32, 0x32, 0x81, 0xff, 0x02, 0xe0, 0x82, 0x84, 0x82, 0xba, + 0x17, 0x44, 0x98, 0xa9, 0x0a, 0xaa, 0xa5, 0xec, 0xbc, 0xce, 0x34, 0x85, 0x2c, 0x8f, 0x86, 0x2e, + 0xd1, 0xe5, 0x74, 0x54, 0xd6, 0x99, 0xa6, 0xc0, 0x0f, 0x17, 0x76, 0xae, 0x29, 0x3b, 0xff, 0x5d, + 0x69, 0xe7, 0x42, 0xec, 0x72, 0x37, 0xf1, 0x13, 0xd8, 0xe0, 0xd2, 0xf2, 0x8b, 0x08, 0x40, 0xb9, + 0x99, 0x51, 0x2d, 0xb7, 0xee, 0xb2, 0x21, 0x2d, 0x6c, 0x9d, 0x2f, 0x52, 0x94, 0x63, 0x0c, 0x69, + 0x12, 0x7a, 0xdc, 0xcc, 0x56, 0x52, 0xb5, 0xbc, 0xad, 0x62, 0xb9, 0x24, 0x1a, 0x8c, 0xcd, 0x9c, + 0x4a, 0xc9, 0x10, 0xbf, 0x44, 0x90, 0xef, 0x13, 0x41, 0x03, 0xc7, 0xa7, 0xdc, 0xcc, 0xab, 0xa1, + 0x8d, 0xbb, 0xe8, 0x78, 0x34, 0x03, 0x29, 0x25, 0xad, 0x5d, 0x49, 0xe1, 0xf9, 0x87, 0xf2, 0xb6, + 0xe7, 0x8b, 0xee, 0xa8, 0x63, 0x39, 0x6c, 0x50, 0x9f, 0x77, 0x71, 0x3b, 0x8b, 0xb8, 0x3e, 0xec, + 0x79, 0xf5, 0xf8, 0x5e, 0xdb, 0x07, 0xf6, 0x82, 0x01, 0x7e, 0x81, 0x20, 0x47, 0x1c, 0xe1, 0x8f, + 0x7d, 0x31, 0x31, 0xe1, 0x96, 0x5b, 0x1a, 0xa3, 0xd3, 0xd4, 0x98, 0x9f, 0xc0, 0x66, 0x3e, 0xff, + 0x47, 0xee, 0x78, 0x71, 0x0f, 0x36, 0x6e, 0x5a, 0x14, 0x47, 0xaf, 0x2d, 0x41, 0xa7, 0xe2, 0x68, + 0x07, 0x7e, 0x3d, 0xa1, 0xe2, 0x19, 0x0b, 0x7b, 0x33, 0x61, 0xb8, 0x08, 0x39, 0x3f, 0x70, 0xd8, + 0xc0, 0x0f, 0x3c, 0xd5, 0x23, 0x65, 0xcf, 0xcf, 0xb2, 0xc6, 0x46, 0xc2, 0x63, 0xb2, 0x16, 0xf5, + 0x9a, 0x9f, 0xb1, 0x09, 0xd9, 0xc8, 0xde, 0x89, 0xbe, 0xe1, 0xb3, 0x63, 0x71, 0x08, 0xeb, 0x37, + 0x6c, 0x5b, 0xc2, 0xb0, 0x1d, 0x67, 0x68, 0x34, 0xb6, 0xef, 0xb2, 0x8a, 0x6f, 0x88, 0xc7, 0x5f, + 0xfc, 0xd7, 0x49, 0x30, 0x1e, 0x50, 0xd2, 0x17, 0xdd, 0x66, 0x9f, 0x86, 0x02, 0x77, 0x20, 0x17, + 0x5d, 0x7b, 0xdf, 0x8d, 0xa6, 0xb6, 0xee, 0xcb, 0xcd, 0xbd, 0xbf, 0x2e, 0xff, 0xff, 0x5d, 0x9b, + 0x53, 0x2f, 0x43, 0xfb, 0x60, 0x7a, 0x5d, 0xce, 0xea, 0xd0, 0xce, 0xaa, 0xc6, 0x6d, 0x17, 0xb7, + 0x21, 0xe7, 0x10, 0x41, 0x3d, 0x16, 0x4e, 0x94, 0x8a, 0x8d, 0xc6, 0x3f, 0x2b, 0x55, 0xc4, 0xb8, + 0x59, 0xfb, 0x1a, 0x64, 0xcf, 0xe1, 0xb8, 0x02, 0x86, 0xab, 0xbf, 0x42, 0x3e, 0x0b, 0x94, 0x9d, + 0x79, 0x3b, 0x9e, 0x5a, 0x6c, 0x34, 0x1d, 0xbb, 0x0f, 0xd5, 0x2d, 0xc8, 0xcd, 0xba, 0x61, 0x03, + 0xb2, 0xc7, 0x87, 0x67, 0x76, 0x7b, 0xff, 0xb4, 0x90, 0x90, 0x87, 0x93, 0xc3, 0xb3, 0xf3, 0xc7, + 0xf6, 0x51, 0x01, 0x55, 0xcf, 0xe1, 0xb7, 0x68, 0xfe, 0x7e, 0x97, 0x3a, 0x3d, 0x9b, 0xf2, 0x51, + 0x5f, 0xe0, 0x16, 0x64, 0x88, 0xa4, 0xc3, 0x4d, 0x74, 0xcb, 0x07, 0x21, 0xc6, 0x5d, 0x7f, 0x10, + 0x34, 0xb2, 0x55, 0xb9, 0xfa, 0x54, 0x4a, 0x5c, 0x4d, 0x4b, 0xe8, 0xcd, 0xb4, 0x84, 0xde, 0x4d, + 0x4b, 0xe8, 0xe3, 0xb4, 0x84, 0x5e, 0x7d, 0x2e, 0x25, 0x9e, 0x66, 0x22, 0x6c, 0x27, 0xa3, 0xfe, + 0x6e, 0xb6, 0xbf, 0x06, 0x00, 0x00, 0xff, 0xff, 0xad, 0xdb, 0xc0, 0x82, 0xe5, 0x06, 0x00, 0x00, } diff --git a/pkg/server/status/status.proto b/pkg/server/status/status.proto index 3962ff2a9da5..ee3952817579 100644 --- a/pkg/server/status/status.proto +++ b/pkg/server/status/status.proto @@ -58,3 +58,28 @@ message NodeStatus { (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" ]; } + +// A HealthAlert is an undesired condition detected by a server which should be +// exposed to the operators. +message HealthAlert { + // store_id is zero for alerts not specific to a store (i.e. apply at the node level). + int32 store_id = 1 [ + // NB: trying to make this nullable does not work with the custom type. You need a + // pointer type as the custom type, but that breaks protoc-gen-gogoroach. + (gogoproto.nullable) = false, + (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/roachpb.StoreID", + (gogoproto.customname) = "StoreID" + ]; + enum Category { + METRICS = 0; + NETWORK = 1; + } + Category category = 2; + string description = 3; + double value = 4; +} + +// HealthCheckResult holds a number of HealthAlerts. +message HealthCheckResult{ + repeated HealthAlert alerts = 1 [(gogoproto.nullable) = false]; +} diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 2091c8eb0fe7..d7e72a7d2c02 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -450,7 +450,7 @@ func (ts *TestServer) Addr() string { // WriteSummaries implements TestServerInterface. func (ts *TestServer) WriteSummaries() error { - return ts.node.writeSummaries(context.TODO()) + return ts.node.writeNodeStatus(context.TODO(), time.Hour) } // AdminURL implements TestServerInterface. diff --git a/pkg/server/updates.go b/pkg/server/updates.go index b70b91ffe132..abdd44009a53 100644 --- a/pkg/server/updates.go +++ b/pkg/server/updates.go @@ -257,7 +257,7 @@ func (s *Server) maybeReportDiagnostics( func (s *Server) getReportingInfo(ctx context.Context) *diagnosticspb.DiagnosticReport { info := diagnosticspb.DiagnosticReport{} - n := s.node.recorder.GetStatusSummary(ctx) + n := s.node.recorder.GenerateNodeStatus(ctx) info.Node = diagnosticspb.NodeInfo{NodeID: s.node.Descriptor.NodeID} secret := sql.ClusterSecret.Get(&s.cfg.Settings.SV) diff --git a/pkg/server/updates_test.go b/pkg/server/updates_test.go index e90124418b45..01485f851167 100644 --- a/pkg/server/updates_test.go +++ b/pkg/server/updates_test.go @@ -265,7 +265,7 @@ func TestReportUsage(t *testing.T) { testutils.SucceedsSoon(t, func() error { expectedUsageReports++ - node := ts.node.recorder.GetStatusSummary(ctx) + node := ts.node.recorder.GenerateNodeStatus(ctx) ts.reportDiagnostics(ctx, 0) keyCounts := make(map[roachpb.StoreID]int64) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index cf201c648ddc..c89fe028308b 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/server/status" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/jobs" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" @@ -62,6 +63,7 @@ var crdbInternal = virtualSchema{ crdbInternalCreateStmtsTable, crdbInternalForwardDependenciesTable, crdbInternalGossipNodesTable, + crdbInternalGossipAlertsTable, crdbInternalGossipLivenessTable, crdbInternalIndexColumnsTable, crdbInternalJobsTable, @@ -872,9 +874,9 @@ func populateSessionsTable( // current node. var crdbInternalLocalMetricsTable = virtualSchemaTable{ schema: `CREATE TABLE crdb_internal.node_metrics ( - store_id INT NULL, -- the store, if any, to which this metric belongs - name STRING, -- name of the metric - value FLOAT -- value of the metric + store_id INT NULL, -- the store, if any, for this metric + name STRING NOT NULL, -- name of the metric + value FLOAT NOT NULL -- value of the metric );`, populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { @@ -886,7 +888,7 @@ var crdbInternalLocalMetricsTable = virtualSchemaTable{ if mr == nil { return nil } - nodeStatus := mr.GetStatusSummary(ctx) + nodeStatus := mr.GenerateNodeStatus(ctx) for i := 0; i <= len(nodeStatus.StoreStatuses); i++ { storeID := tree.DNull mtr := nodeStatus.Metrics @@ -1806,6 +1808,70 @@ CREATE TABLE crdb_internal.gossip_liveness ( }, } +// crdbInternalGossipAlertsTable exposes current health alerts in the cluster. +var crdbInternalGossipAlertsTable = virtualSchemaTable{ + schema: ` +CREATE TABLE crdb_internal.gossip_alerts ( + node_id INT NOT NULL, + store_id INT NULL, -- null for alerts not associated to a store + category STRING NOT NULL, -- type of alert, usually by subsystem + description STRING NOT NULL, -- name of the alert (depends on subsystem) + value FLOAT NOT NULL -- value of the alert (depends on subsystem, can be NaN) +) + `, + populate: func(ctx context.Context, p *planner, _ *DatabaseDescriptor, addRow func(...tree.Datum) error) error { + if err := p.RequireSuperUser(ctx, "read crdb_internal.gossip_alerts "); err != nil { + return err + } + + g := p.ExecCfg().Gossip + + type resultWithNodeID struct { + roachpb.NodeID + status.HealthCheckResult + } + var results []resultWithNodeID + if err := g.IterateInfos(gossip.KeyNodeHealthAlertPrefix, func(key string, i gossip.Info) error { + bytes, err := i.Value.GetBytes() + if err != nil { + return errors.Wrapf(err, "failed to extract bytes for key %q", key) + } + + var d status.HealthCheckResult + if err := protoutil.Unmarshal(bytes, &d); err != nil { + return errors.Wrapf(err, "failed to parse value for key %q", key) + } + nodeID, err := gossip.NodeIDFromKey(key) + if err != nil { + return errors.Wrapf(err, "failed to parse node ID from key %q", key) + } + results = append(results, resultWithNodeID{nodeID, d}) + return nil + }); err != nil { + return err + } + + for _, result := range results { + for _, alert := range result.Alerts { + storeID := tree.DNull + if alert.StoreID != 0 { + storeID = tree.NewDInt(tree.DInt(alert.StoreID)) + } + if err := addRow( + tree.NewDInt(tree.DInt(result.NodeID)), + storeID, + tree.NewDString(strings.ToLower(alert.Category.String())), + tree.NewDString(alert.Description), + tree.NewDFloat(tree.DFloat(alert.Value)), + ); err != nil { + return err + } + } + } + return nil + }, +} + func addPartitioningRows( table *sqlbase.TableDescriptor, index *sqlbase.IndexDescriptor, diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index dec8819a9671..22a578aa36b4 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -19,6 +19,7 @@ cluster_sessions cluster_settings create_statements forward_dependencies +gossip_alerts gossip_liveness gossip_nodes index_columns @@ -324,6 +325,9 @@ select * from crdb_internal.kv_node_status query error pq: only superusers are allowed to read crdb_internal.kv_store_status select * from crdb_internal.kv_store_status +query error pq: only superusers are allowed to read crdb_internal.gossip_alerts +select * from crdb_internal.gossip_alerts + # Anyone can see the executable version. query T select crdb_internal.node_executable_version() diff --git a/pkg/sql/logictest/testdata/logic_test/explain b/pkg/sql/logictest/testdata/logic_test/explain index 1247a1e45718..ac50b61beb67 100644 --- a/pkg/sql/logictest/testdata/logic_test/explain +++ b/pkg/sql/logictest/testdata/logic_test/explain @@ -188,7 +188,7 @@ sort · · └── render · · └── filter · · └── values · · -· size 6 columns, 82 rows +· size 6 columns, 83 rows query TTT EXPLAIN SHOW DATABASE @@ -251,7 +251,7 @@ sort · · ├── render · · │ └── filter · · │ └── values · · - │ size 17 columns, 747 rows + │ size 17 columns, 752 rows └── render · · └── filter · · └── values · · @@ -265,7 +265,7 @@ sort · · └── render · · └── filter · · └── values · · -· size 8 columns, 373 rows +· size 8 columns, 376 rows query TTT diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 617bbcb22284..bb6a98a1f9e2 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -40,6 +40,7 @@ test crdb_internal cluster_sessions public SELECT test crdb_internal cluster_settings public SELECT test crdb_internal create_statements public SELECT test crdb_internal forward_dependencies public SELECT +test crdb_internal gossip_alerts public SELECT test crdb_internal gossip_liveness public SELECT test crdb_internal gossip_nodes public SELECT test crdb_internal index_columns public SELECT @@ -148,6 +149,7 @@ a crdb_internal cluster_sessions public SELE a crdb_internal cluster_settings public SELECT a crdb_internal create_statements public SELECT a crdb_internal forward_dependencies public SELECT +a crdb_internal gossip_alerts public SELECT a crdb_internal gossip_liveness public SELECT a crdb_internal gossip_nodes public SELECT a crdb_internal index_columns public SELECT @@ -231,10 +233,10 @@ a pg_catalog pg_views public SELE a public NULL admin ALL a public NULL readwrite ALL a public NULL root ALL -system crdb_internal NULL admin SELECT system crdb_internal NULL admin GRANT -system crdb_internal NULL root SELECT +system crdb_internal NULL admin SELECT system crdb_internal NULL root GRANT +system crdb_internal NULL root SELECT system crdb_internal backward_dependencies public SELECT system crdb_internal builtin_functions public SELECT system crdb_internal cluster_queries public SELECT @@ -242,6 +244,7 @@ system crdb_internal cluster_sessions public SELE system crdb_internal cluster_settings public SELECT system crdb_internal create_statements public SELECT system crdb_internal forward_dependencies public SELECT +system crdb_internal gossip_alerts public SELECT system crdb_internal gossip_liveness public SELECT system crdb_internal gossip_nodes public SELECT system crdb_internal index_columns public SELECT @@ -328,134 +331,134 @@ system public NULL admin SELE system public NULL admin GRANT system public NULL root GRANT system public NULL root SELECT -system public descriptor admin GRANT system public descriptor admin SELECT +system public descriptor admin GRANT system public descriptor root SELECT system public descriptor root GRANT -system public eventlog admin INSERT +system public eventlog admin SELECT system public eventlog admin GRANT system public eventlog admin DELETE -system public eventlog admin SELECT system public eventlog admin UPDATE -system public eventlog root GRANT +system public eventlog admin INSERT system public eventlog root DELETE -system public eventlog root INSERT system public eventlog root UPDATE system public eventlog root SELECT -system public jobs admin UPDATE +system public eventlog root GRANT +system public eventlog root INSERT system public jobs admin SELECT -system public jobs admin INSERT -system public jobs admin GRANT +system public jobs admin UPDATE system public jobs admin DELETE -system public jobs root UPDATE +system public jobs admin GRANT +system public jobs admin INSERT system public jobs root DELETE +system public jobs root SELECT system public jobs root GRANT +system public jobs root UPDATE system public jobs root INSERT -system public jobs root SELECT -system public lease admin UPDATE +system public lease admin GRANT system public lease admin SELECT +system public lease admin UPDATE system public lease admin DELETE -system public lease admin GRANT system public lease admin INSERT system public lease root INSERT system public lease root SELECT -system public lease root UPDATE system public lease root GRANT system public lease root DELETE -system public locations admin DELETE +system public lease root UPDATE system public locations admin GRANT system public locations admin INSERT system public locations admin SELECT +system public locations admin DELETE system public locations admin UPDATE -system public locations root INSERT +system public locations root DELETE system public locations root GRANT -system public locations root UPDATE +system public locations root INSERT system public locations root SELECT -system public locations root DELETE +system public locations root UPDATE system public namespace admin GRANT system public namespace admin SELECT system public namespace root SELECT system public namespace root GRANT -system public rangelog admin DELETE -system public rangelog admin GRANT -system public rangelog admin INSERT system public rangelog admin SELECT +system public rangelog admin INSERT system public rangelog admin UPDATE -system public rangelog root DELETE -system public rangelog root SELECT +system public rangelog admin GRANT +system public rangelog admin DELETE system public rangelog root GRANT system public rangelog root INSERT +system public rangelog root DELETE system public rangelog root UPDATE -system public role_members admin GRANT +system public rangelog root SELECT system public role_members admin INSERT -system public role_members admin UPDATE system public role_members admin SELECT +system public role_members admin UPDATE system public role_members admin DELETE -system public role_members root DELETE +system public role_members admin GRANT +system public role_members root INSERT +system public role_members root SELECT system public role_members root UPDATE +system public role_members root DELETE system public role_members root GRANT -system public role_members root SELECT -system public role_members root INSERT -system public settings admin GRANT -system public settings admin INSERT +system public settings admin UPDATE system public settings admin DELETE +system public settings admin INSERT system public settings admin SELECT -system public settings admin UPDATE +system public settings admin GRANT system public settings root SELECT -system public settings root DELETE -system public settings root UPDATE system public settings root GRANT system public settings root INSERT +system public settings root UPDATE +system public settings root DELETE +system public table_statistics admin GRANT system public table_statistics admin UPDATE -system public table_statistics admin DELETE system public table_statistics admin INSERT system public table_statistics admin SELECT -system public table_statistics admin GRANT -system public table_statistics root SELECT +system public table_statistics admin DELETE +system public table_statistics root GRANT system public table_statistics root UPDATE +system public table_statistics root SELECT system public table_statistics root DELETE -system public table_statistics root GRANT system public table_statistics root INSERT +system public ui admin DELETE system public ui admin SELECT system public ui admin GRANT -system public ui admin DELETE -system public ui admin UPDATE system public ui admin INSERT -system public ui root UPDATE +system public ui admin UPDATE +system public ui root DELETE system public ui root GRANT -system public ui root INSERT system public ui root SELECT -system public ui root DELETE -system public users admin DELETE -system public users admin GRANT +system public ui root INSERT +system public ui root UPDATE system public users admin SELECT +system public users admin GRANT +system public users admin DELETE system public users admin UPDATE system public users admin INSERT -system public users root INSERT system public users root DELETE -system public users root GRANT -system public users root SELECT +system public users root INSERT system public users root UPDATE -system public web_sessions admin GRANT -system public web_sessions admin INSERT -system public web_sessions admin SELECT -system public web_sessions admin UPDATE +system public users root SELECT +system public users root GRANT system public web_sessions admin DELETE -system public web_sessions root DELETE -system public web_sessions root SELECT -system public web_sessions root INSERT +system public web_sessions admin UPDATE +system public web_sessions admin SELECT +system public web_sessions admin INSERT +system public web_sessions admin GRANT system public web_sessions root UPDATE +system public web_sessions root INSERT system public web_sessions root GRANT -system public zones admin UPDATE +system public web_sessions root DELETE +system public web_sessions root SELECT system public zones admin INSERT system public zones admin DELETE -system public zones admin SELECT system public zones admin GRANT -system public zones root UPDATE -system public zones root INSERT -system public zones root GRANT +system public zones admin UPDATE +system public zones admin SELECT system public zones root DELETE +system public zones root GRANT +system public zones root UPDATE system public zones root SELECT +system public zones root INSERT test crdb_internal NULL admin ALL test crdb_internal NULL root ALL test crdb_internal backward_dependencies public SELECT @@ -465,6 +468,7 @@ test crdb_internal cluster_sessions public SELE test crdb_internal cluster_settings public SELECT test crdb_internal create_statements public SELECT test crdb_internal forward_dependencies public SELECT +test crdb_internal gossip_alerts public SELECT test crdb_internal gossip_liveness public SELECT test crdb_internal gossip_nodes public SELECT test crdb_internal index_columns public SELECT diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 6158ca0fb53e..a2ccb22c4433 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -212,6 +212,7 @@ crdb_internal cluster_sessions crdb_internal cluster_settings crdb_internal create_statements crdb_internal forward_dependencies +crdb_internal gossip_alerts crdb_internal gossip_liveness crdb_internal gossip_nodes crdb_internal index_columns @@ -220,9 +221,9 @@ crdb_internal kv_node_status crdb_internal kv_store_status crdb_internal leases crdb_internal node_build_info +crdb_internal node_metrics crdb_internal node_queries crdb_internal node_runtime_info -crdb_internal node_metrics crdb_internal node_sessions crdb_internal node_statement_statistics crdb_internal partitions @@ -328,6 +329,7 @@ cluster_sessions cluster_settings create_statements forward_dependencies +gossip_alerts gossip_liveness gossip_nodes index_columns @@ -451,6 +453,7 @@ system crdb_internal cluster_sessions SYSTEM VIE system crdb_internal cluster_settings SYSTEM VIEW NO 1 system crdb_internal create_statements SYSTEM VIEW NO 1 system crdb_internal forward_dependencies SYSTEM VIEW NO 1 +system crdb_internal gossip_alerts SYSTEM VIEW NO 1 system crdb_internal gossip_liveness SYSTEM VIEW NO 1 system crdb_internal gossip_nodes SYSTEM VIEW NO 1 system crdb_internal index_columns SYSTEM VIEW NO 1 @@ -963,64 +966,65 @@ NULL public system information_schema column_privileges NULL public system information_schema columns SELECT NULL NULL NULL public system information_schema constraint_column_usage SELECT NULL NULL NULL public system crdb_internal create_statements SELECT NULL NULL -NULL root system public descriptor GRANT NULL NULL +NULL admin system public descriptor GRANT NULL NULL NULL root system public descriptor SELECT NULL NULL NULL admin system public descriptor SELECT NULL NULL -NULL admin system public descriptor GRANT NULL NULL +NULL root system public descriptor GRANT NULL NULL NULL public system information_schema enabled_roles SELECT NULL NULL -NULL root system public eventlog DELETE NULL NULL -NULL root system public eventlog SELECT NULL NULL +NULL admin system public eventlog DELETE NULL NULL NULL admin system public eventlog GRANT NULL NULL +NULL root system public eventlog DELETE NULL NULL +NULL root system public eventlog GRANT NULL NULL NULL root system public eventlog INSERT NULL NULL +NULL root system public eventlog SELECT NULL NULL +NULL admin system public eventlog UPDATE NULL NULL NULL root system public eventlog UPDATE NULL NULL -NULL root system public eventlog GRANT NULL NULL -NULL admin system public eventlog DELETE NULL NULL -NULL admin system public eventlog INSERT NULL NULL NULL admin system public eventlog SELECT NULL NULL -NULL admin system public eventlog UPDATE NULL NULL +NULL admin system public eventlog INSERT NULL NULL NULL public system crdb_internal forward_dependencies SELECT NULL NULL +NULL public system crdb_internal gossip_alerts SELECT NULL NULL NULL public system crdb_internal gossip_liveness SELECT NULL NULL NULL public system crdb_internal gossip_nodes SELECT NULL NULL NULL public system crdb_internal index_columns SELECT NULL NULL -NULL root system public jobs SELECT NULL NULL +NULL admin system public jobs GRANT NULL NULL NULL admin system public jobs DELETE NULL NULL -NULL admin system public jobs INSERT NULL NULL -NULL admin system public jobs SELECT NULL NULL -NULL admin system public jobs UPDATE NULL NULL -NULL root system public jobs DELETE NULL NULL NULL root system public jobs GRANT NULL NULL NULL root system public jobs INSERT NULL NULL NULL public system crdb_internal jobs SELECT NULL NULL +NULL root system public jobs SELECT NULL NULL +NULL admin system public jobs INSERT NULL NULL +NULL root system public jobs DELETE NULL NULL +NULL admin system public jobs UPDATE NULL NULL NULL root system public jobs UPDATE NULL NULL -NULL admin system public jobs GRANT NULL NULL +NULL admin system public jobs SELECT NULL NULL NULL public system information_schema key_column_usage SELECT NULL NULL NULL public system crdb_internal kv_node_status SELECT NULL NULL NULL public system crdb_internal kv_store_status SELECT NULL NULL +NULL root system public lease SELECT NULL NULL +NULL admin system public lease GRANT NULL NULL +NULL admin system public lease DELETE NULL NULL NULL admin system public lease INSERT NULL NULL NULL admin system public lease SELECT NULL NULL -NULL root system public lease SELECT NULL NULL NULL admin system public lease UPDATE NULL NULL NULL root system public lease DELETE NULL NULL -NULL admin system public lease GRANT NULL NULL NULL root system public lease GRANT NULL NULL -NULL root system public lease INSERT NULL NULL NULL root system public lease UPDATE NULL NULL -NULL admin system public lease DELETE NULL NULL +NULL root system public lease INSERT NULL NULL NULL public system crdb_internal leases SELECT NULL NULL +NULL root system public locations GRANT NULL NULL +NULL admin system public locations UPDATE NULL NULL NULL admin system public locations SELECT NULL NULL -NULL root system public locations DELETE NULL NULL -NULL root system public locations INSERT NULL NULL NULL admin system public locations DELETE NULL NULL -NULL root system public locations GRANT NULL NULL -NULL admin system public locations GRANT NULL NULL NULL admin system public locations INSERT NULL NULL +NULL admin system public locations GRANT NULL NULL NULL root system public locations UPDATE NULL NULL NULL root system public locations SELECT NULL NULL -NULL admin system public locations UPDATE NULL NULL +NULL root system public locations INSERT NULL NULL +NULL root system public locations DELETE NULL NULL NULL root system public namespace SELECT NULL NULL NULL root system public namespace GRANT NULL NULL -NULL admin system public namespace SELECT NULL NULL NULL admin system public namespace GRANT NULL NULL +NULL admin system public namespace SELECT NULL NULL NULL public system crdb_internal node_build_info SELECT NULL NULL NULL public system crdb_internal node_metrics SELECT NULL NULL NULL public system crdb_internal node_queries SELECT NULL NULL @@ -1173,6 +1177,7 @@ NULL public system crdb_internal cluster_sessions NULL public system crdb_internal cluster_settings SELECT NULL NULL NULL public system crdb_internal create_statements SELECT NULL NULL NULL public system crdb_internal forward_dependencies SELECT NULL NULL +NULL public system crdb_internal gossip_alerts SELECT NULL NULL NULL public system crdb_internal gossip_liveness SELECT NULL NULL NULL public system crdb_internal gossip_nodes SELECT NULL NULL NULL public system crdb_internal index_columns SELECT NULL NULL diff --git a/pkg/storage/metrics.go b/pkg/storage/metrics.go index 2778cc501094..b07ea300ce70 100644 --- a/pkg/storage/metrics.go +++ b/pkg/storage/metrics.go @@ -561,7 +561,7 @@ type StoreMetrics struct { RdbNumSSTables *metric.Gauge // TODO(mrtracy): This should be removed as part of #4465. This is only - // maintained to keep the current structure of StatusSummaries; it would be + // maintained to keep the current structure of NodeStatus; it would be // better to convert the Gauges above into counters which are adjusted // accordingly. @@ -829,8 +829,8 @@ func newStoreMetrics(histogramWindow time.Duration) *StoreMetrics { SplitQueuePending: metric.NewGauge(metaSplitQueuePending), SplitQueueProcessingNanos: metric.NewCounter(metaSplitQueueProcessingNanos), SplitQueuePurgatory: metric.NewGauge(metaSplitQueuePurgatory), - TimeSeriesMaintenanceQueueSuccesses: metric.NewCounter(metaTimeSeriesMaintenanceQueueFailures), - TimeSeriesMaintenanceQueueFailures: metric.NewCounter(metaTimeSeriesMaintenanceQueueSuccesses), + TimeSeriesMaintenanceQueueSuccesses: metric.NewCounter(metaTimeSeriesMaintenanceQueueSuccesses), + TimeSeriesMaintenanceQueueFailures: metric.NewCounter(metaTimeSeriesMaintenanceQueueFailures), TimeSeriesMaintenanceQueuePending: metric.NewGauge(metaTimeSeriesMaintenanceQueuePending), TimeSeriesMaintenanceQueueProcessingNanos: metric.NewCounter(metaTimeSeriesMaintenanceQueueProcessingNanos), diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 4fd73edfd0dd..f2d491c467d2 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -2114,7 +2114,7 @@ func splitPostApply( // TODO(peter): In single-node clusters, we enqueue the right-hand side of // the split (the new range) for Raft processing so that the corresponding // Raft group is created. This shouldn't be necessary for correctness, but - // some tests rely on this (e.g. server.TestStatusSummaries). + // some tests rely on this (e.g. server.TestNodeStatusWritten). r.store.enqueueRaftUpdateCheck(rightRng.RangeID) } } @@ -2579,7 +2579,7 @@ func (s *Store) Metrics() *StoreMetrics { // MVCCStats returns the current MVCCStats accumulated for this store. // TODO(mrtracy): This should be removed as part of #4465, this is only needed -// to support the current StatusSummary structures which will be changing. +// to support the current NodeStatus structures which will be changing. func (s *Store) MVCCStats() enginepb.MVCCStats { s.metrics.mu.Lock() defer s.metrics.mu.Unlock()