Skip to content

Commit

Permalink
Merge #135705
Browse files Browse the repository at this point in the history
135705: server,ui: include recorded tsdb metric names in /_admin/v1/metricmet… r=kyle-a-wong a=kyle-a-wong

…adata

Added an additional field, `recordedNames` to the `/_admin/v1/metricdata` response that provides a mapping from metric name to the name recorded in tsdb. This will enable the ability to create tsdb queries purely with the response of this API, as opposed to needing additional data from elsewhere. Metrics recorded in tsdb are prefixed with either `cr.node.` or `cr.store.` depending on if they are node or store level metrics.

The custom chart component in db-console is an example of where this was needed. Previously, the component requires both data from this API and metric data from `/_status/nodes_ui` in order to create a list of metrics to build custom queries with. Now, the custom chart component can create this list of metrics purely with the data from `/_admin/v1/metricdata`

Epic: none
Release note: none

Co-authored-by: Kyle Wong <[email protected]>
  • Loading branch information
craig[bot] and kyle-a-wong committed Nov 22, 2024
2 parents 0b07e2a + 40e50af commit 2a92c11
Show file tree
Hide file tree
Showing 8 changed files with 114 additions and 51 deletions.
15 changes: 15 additions & 0 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -7271,6 +7271,7 @@ MetricMetadataResponse contains the metadata for all metrics.
| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| metadata | [MetricMetadataResponse.MetadataEntry](#cockroach.server.serverpb.MetricMetadataResponse-cockroach.server.serverpb.MetricMetadataResponse.MetadataEntry) | repeated | | [reserved](#support-status) |
| recordedNames | [MetricMetadataResponse.RecordedNamesEntry](#cockroach.server.serverpb.MetricMetadataResponse-cockroach.server.serverpb.MetricMetadataResponse.RecordedNamesEntry) | repeated | Maps of metric metadata names to the tsdb recorded metric names | [reserved](#support-status) |



Expand All @@ -7291,6 +7292,20 @@ MetricMetadataResponse contains the metadata for all metrics.



<a name="cockroach.server.serverpb.MetricMetadataResponse-cockroach.server.serverpb.MetricMetadataResponse.RecordedNamesEntry"></a>
#### MetricMetadataResponse.RecordedNamesEntry



| Field | Type | Label | Description | Support status |
| ----- | ---- | ----- | ----------- | -------------- |
| key | [string](#cockroach.server.serverpb.MetricMetadataResponse-string) | | | |
| value | [string](#cockroach.server.serverpb.MetricMetadataResponse-string) | | | |






## ChartCatalog

Expand Down
4 changes: 3 additions & 1 deletion pkg/server/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -301,8 +301,10 @@ func (s *adminServer) AllMetricMetadata(
) (*serverpb.MetricMetadataResponse, error) {

md, _, _ := s.metricsRecorder.GetMetricsMetadata(true /* combine */)
metricNames := s.metricsRecorder.GetRecordedMetricNames(md)
resp := &serverpb.MetricMetadataResponse{
Metadata: md,
Metadata: md,
RecordedNames: metricNames,
}

return resp, nil
Expand Down
14 changes: 14 additions & 0 deletions pkg/server/application_api/metrics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,20 @@ func TestMetricsMetadata(t *testing.T) {
}
}

func TestGetRecordedMetricNames(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
s := serverutils.StartServerOnly(t, base.TestServerArgs{})
defer s.Stopper().Stop(context.Background())
metricsMetadata, _, _ := s.MetricsRecorder().GetMetricsMetadata(true /* combine */)
recordedNames := s.MetricsRecorder().GetRecordedMetricNames(metricsMetadata)

require.Equal(t, len(metricsMetadata), len(recordedNames))
for _, v := range recordedNames {
require.True(t, strings.HasPrefix(v, "cr.node") || strings.HasPrefix(v, "cr.store"))
}
}

// TestStatusVars verifies that prometheus metrics are available via the
// /_status/vars and /_status/load endpoints.
func TestStatusVars(t *testing.T) {
Expand Down
2 changes: 2 additions & 0 deletions pkg/server/serverpb/admin.proto
Original file line number Diff line number Diff line change
Expand Up @@ -846,6 +846,8 @@ message MetricMetadataRequest {
// MetricMetadataResponse contains the metadata for all metrics.
message MetricMetadataResponse {
map<string, cockroach.util.metric.Metadata> metadata = 1 [(gogoproto.nullable) = false];
// Maps of metric metadata names to the tsdb recorded metric names
map<string, string> recordedNames = 2 [(gogoproto.nullable) = false];
}

message EnqueueRangeRequest {
Expand Down
42 changes: 31 additions & 11 deletions pkg/server/status/recorder.go
Original file line number Diff line number Diff line change
Expand Up @@ -486,22 +486,42 @@ func (mr *MetricsRecorder) GetMetricsMetadata(
mr.mu.logRegistry.WriteMetricsMetadata(srvMetrics)
mr.mu.sysRegistry.WriteMetricsMetadata(srvMetrics)

// Get a random storeID.
var sID roachpb.StoreID
mr.writeStoreMetricsMetadata(nodeMetrics)
return nodeMetrics, appMetrics, srvMetrics
}

// GetRecordedMetricNames takes a map of metric metadata and returns a map
// of the metadata name to the name the metric is recorded with in tsdb.
func (mr *MetricsRecorder) GetRecordedMetricNames(
allMetadata map[string]metric.Metadata,
) map[string]string {
storeMetricsMap := make(map[string]metric.Metadata)
tsDbMetricNames := make(map[string]string, len(allMetadata))
mr.writeStoreMetricsMetadata(storeMetricsMap)
for k := range allMetadata {
prefix := nodeTimeSeriesPrefix
if _, ok := storeMetricsMap[k]; ok {
prefix = storeTimeSeriesPrefix
}

storeFound := false
for storeID := range mr.mu.storeRegistries {
sID = storeID
storeFound = true
break
tsDbMetricNames[k] = fmt.Sprintf(prefix, k)
}
return tsDbMetricNames
}

// Get metric metadata from that store because all stores have the same metadata.
if storeFound {
mr.mu.storeRegistries[sID].WriteMetricsMetadata(nodeMetrics)
// writeStoreMetricsMetadata Gets a store from mr.mu.storeRegistries and writes
// the metrics metadata to the provided map.
func (mr *MetricsRecorder) writeStoreMetricsMetadata(metricsMetadata map[string]metric.Metadata) {
if len(mr.mu.storeRegistries) == 0 {
return
}

return nodeMetrics, appMetrics, srvMetrics
// All store registries should have the same metadata, so only the metadata
// from the first store is used to write to metricsMetadata.
for _, registry := range mr.mu.storeRegistries {
registry.WriteMetricsMetadata(metricsMetadata)
return
}
}

// getNetworkActivity produces a map of network activity from this node to all
Expand Down
5 changes: 2 additions & 3 deletions pkg/ui/workspaces/db-console/src/redux/metricMetadata.ts
Original file line number Diff line number Diff line change
Expand Up @@ -8,14 +8,13 @@ import { createSelector } from "reselect";
import { AdminUIState } from "src/redux/state";
import { MetricMetadataResponseMessage } from "src/util/api";

export type MetricsMetadata = MetricMetadataResponseMessage["metadata"];
export type MetricsMetadata = MetricMetadataResponseMessage;

// State selectors
const metricsMetadataStateSelector = (state: AdminUIState) =>
state.cachedData.metricMetadata.data;

export const metricsMetadataSelector = createSelector(
metricsMetadataStateSelector,
(metricsMetadata): MetricsMetadata =>
metricsMetadata ? metricsMetadata.metadata : undefined,
(metricsMetadata): MetricsMetadata => metricsMetadata,
);
Original file line number Diff line number Diff line change
Expand Up @@ -4,26 +4,35 @@
// included in the /LICENSE file.

import * as protos from "src/js/protos";
import { MetricsMetadata } from "src/redux/metricMetadata";
import { NodesSummary } from "src/redux/nodes";
import { INodeStatus } from "src/util/proto";
import { CustomMetricState } from "src/views/reports/containers/customChart/customMetric";
import { GetSources } from "src/views/reports/containers/customChart/index";
import { getSources } from "src/views/reports/containers/customChart/index";

import TimeSeriesQueryAggregator = protos.cockroach.ts.tspb.TimeSeriesQueryAggregator;
import TimeSeriesQueryDerivative = protos.cockroach.ts.tspb.TimeSeriesQueryDerivative;

const emptyMetricsMetadata: MetricsMetadata = {
metadata: {},
recordedNames: {},
};
describe("Custom charts page", function () {
describe("Getting metric sources", function () {
it("returns empty when nodesSummary is undefined", function () {
const metricState = new testCustomMetricStateBuilder().build();
expect(GetSources(undefined, metricState)).toStrictEqual([]);
expect(
getSources(undefined, metricState, emptyMetricsMetadata),
).toStrictEqual([]);
});

it("returns empty when the nodeStatuses collection is empty", function () {
const nodesSummary = new testNodesSummaryBuilder().build();
nodesSummary.nodeStatuses = [];
const metricState = new testCustomMetricStateBuilder().build();
expect(GetSources(nodesSummary, metricState)).toStrictEqual([]);
expect(
getSources(nodesSummary, metricState, emptyMetricsMetadata),
).toStrictEqual([]);
});

it("returns empty when no specific node source is requested, nor per-source metrics", function () {
Expand All @@ -32,7 +41,9 @@ describe("Custom charts page", function () {
.setNodeSource("")
.setIsPerSource(false)
.build();
expect(GetSources(nodesSummary, metricState)).toStrictEqual([]);
expect(
getSources(nodesSummary, metricState, emptyMetricsMetadata),
).toStrictEqual([]);
});

describe("The metric is at the store-level", function () {
Expand All @@ -49,9 +60,9 @@ describe("Custom charts page", function () {
"1": expectedSources,
})
.build();
expect(GetSources(nodesSummary, metricState)).toStrictEqual(
expectedSources,
);
expect(
getSources(nodesSummary, metricState, emptyMetricsMetadata),
).toStrictEqual(expectedSources);
});

it("returns all known store IDs for the cluster when no node source is set", function () {
Expand All @@ -66,7 +77,11 @@ describe("Custom charts page", function () {
"3": ["7", "8", "9"],
})
.build();
const actualSources = GetSources(nodesSummary, metricState).sort();
const actualSources = getSources(
nodesSummary,
metricState,
emptyMetricsMetadata,
).sort();
expect(actualSources).toStrictEqual(expectedSources);
});
});
Expand All @@ -81,9 +96,9 @@ describe("Custom charts page", function () {
.setNodeSource("1")
.build();
const nodesSummary = new testNodesSummaryBuilder().build();
expect(GetSources(nodesSummary, metricState)).toStrictEqual(
expectedSources,
);
expect(
getSources(nodesSummary, metricState, emptyMetricsMetadata),
).toStrictEqual(expectedSources);
});

it("returns all known node IDs when no node source is set", function () {
Expand All @@ -94,9 +109,9 @@ describe("Custom charts page", function () {
const nodesSummary = new testNodesSummaryBuilder()
.setNodeIDs(["1", "2", "3"])
.build();
expect(GetSources(nodesSummary, metricState)).toStrictEqual(
expectedSources,
);
expect(
getSources(nodesSummary, metricState, emptyMetricsMetadata),
).toStrictEqual(expectedSources);
});
});
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,6 @@
import { AxisUnits, TimeScale } from "@cockroachlabs/cluster-ui";
import flatMap from "lodash/flatMap";
import flow from "lodash/flow";
import has from "lodash/has";
import isEmpty from "lodash/isEmpty";
import keys from "lodash/keys";
import map from "lodash/map";
Expand Down Expand Up @@ -76,9 +75,10 @@ interface UrlState {
charts: string;
}

export const GetSources = (
export const getSources = (
nodesSummary: NodesSummary,
metricState: CustomMetricState,
metricsMetadata: MetricsMetadata,
): string[] => {
if (!(nodesSummary?.nodeStatuses?.length > 0)) {
return [];
Expand All @@ -89,7 +89,7 @@ export const GetSources = (
if (metricState.nodeSource === "" && !metricState.perSource) {
return [];
}
if (isStoreMetric(nodesSummary.nodeStatuses[0], metricState.metric)) {
if (isStoreMetric(metricsMetadata.recordedNames, metricState.metric)) {
// If a specific node is selected, return the storeIDs associated with that node.
// Otherwise, we're at the cluster level, so we grab each store ID.
return metricState.nodeSource
Expand Down Expand Up @@ -133,23 +133,18 @@ export class CustomChart extends React.Component<
// Selector which computes dropdown options based on the metrics which are
// currently being stored on the cluster.
private metricOptions = createSelector(
(summary: NodesSummary) => summary.nodeStatuses,
(_summary: NodesSummary, metricsMetadata: MetricsMetadata) =>
metricsMetadata,
(nodeStatuses, metadata = {}): DropdownOption[] => {
if (isEmpty(nodeStatuses)) {
(metricsMetadata: MetricsMetadata) => metricsMetadata,
(metricsMetadata): DropdownOption[] => {
if (isEmpty(metricsMetadata?.metadata)) {
return [];
}

return keys(nodeStatuses[0].metrics).map(k => {
const fullMetricName = isStoreMetric(nodeStatuses[0], k)
? "cr.store." + k
: "cr.node." + k;

return keys(metricsMetadata.metadata).map(k => {
const fullMetricName = metricsMetadata.recordedNames[k];
return {
value: fullMetricName,
label: k,
description: metadata[k] && metadata[k].help,
description: metricsMetadata.metadata[k]?.help,
};
});
},
Expand Down Expand Up @@ -243,7 +238,7 @@ export class CustomChart extends React.Component<
// This function handles the logic related to creating Metric components
// based on perNode and perTenant flags.
renderMetricComponents = (metrics: CustomMetricState[], index: number) => {
const { nodesSummary, tenantOptions } = this.props;
const { nodesSummary, tenantOptions, metricsMetadata } = this.props;
// We require nodes information to determine sources (storeIDs/nodeIDs) down below.
if (!(nodesSummary?.nodeStatuses?.length > 0)) {
return;
Expand All @@ -253,8 +248,8 @@ export class CustomChart extends React.Component<
if (m.metric === "") {
return "";
}
const sources = getSources(nodesSummary, m, metricsMetadata);
if (m.perSource && m.perTenant) {
const sources = GetSources(nodesSummary, m);
return flatMap(sources, source => {
return tenants.map(tenant => (
<Metric
Expand All @@ -270,7 +265,6 @@ export class CustomChart extends React.Component<
));
});
} else if (m.perSource) {
const sources = GetSources(nodesSummary, m);
return map(sources, source => (
<Metric
key={`${index}${i}${source}`}
Expand All @@ -284,7 +278,6 @@ export class CustomChart extends React.Component<
/>
));
} else if (m.perTenant) {
const sources = GetSources(nodesSummary, m);
return tenants.map(tenant => (
<Metric
key={`${index}${i}${tenant.value}`}
Expand All @@ -306,7 +299,7 @@ export class CustomChart extends React.Component<
aggregator={m.aggregator}
downsampler={m.downsampler}
derivative={m.derivative}
sources={GetSources(nodesSummary, m)}
sources={sources}
tenantSource={m.tenantSource}
/>
);
Expand Down Expand Up @@ -356,7 +349,7 @@ export class CustomChart extends React.Component<
<>
{charts.map((chart, i) => (
<CustomChartTable
metricOptions={this.metricOptions(nodesSummary, metricsMetadata)}
metricOptions={this.metricOptions(metricsMetadata)}
nodeOptions={this.nodeOptions(nodesSummary)}
tenantOptions={tenantOptions}
currentTenant={currentTenant}
Expand Down Expand Up @@ -443,9 +436,12 @@ export default withRouter(
connect(mapStateToProps, mapDispatchToProps)(CustomChart),
);

function isStoreMetric(nodeStatus: INodeStatus, metricName: string) {
function isStoreMetric(
recordedNames: Record<string, string>,
metricName: string,
) {
if (metricName?.startsWith("cr.store")) {
return true;
}
return has(nodeStatus.store_statuses[0].metrics, metricName);
return recordedNames[metricName]?.startsWith("cr.store") || false;
}

0 comments on commit 2a92c11

Please sign in to comment.