Skip to content

Commit

Permalink
Use MetricNameBuilder to create labeled metric names
Browse files Browse the repository at this point in the history
  • Loading branch information
JayajP committed Apr 1, 2024
1 parent ebf509e commit ed0dc2e
Showing 1 changed file with 42 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,7 @@
import io.grpc.Status;
import java.time.Instant;
import java.util.List;
import java.util.NavigableMap;
import java.util.Optional;
import java.util.TreeMap;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import org.apache.beam.sdk.io.gcp.bigquery.RetryManager.Operation.Context;
Expand Down Expand Up @@ -133,23 +131,38 @@ public static ParsedMetricName create(String baseName) {
}

/**
* Returns a metric name that merges the baseName with metricLables formatted as.
* Builder class for a labeled {@code MetricName}. Returns a metric name that merges the baseName
* with metricLables formatted as.
*
* <p>'{baseName}-{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};'
*/
private static String createLabeledMetricName(
String baseName, NavigableMap<String, String> metricLabels) {
StringBuilder nameBuilder = new StringBuilder(baseName + METRIC_NAME_DELIMITER);

metricLabels.forEach(
(labelKey, labelVal) ->
nameBuilder.append(labelKey + METRIC_KV_DELIMITER + labelVal + LABEL_DELIMITER));
return nameBuilder.toString();
private static class MetricNameBuilder {
private final StringBuilder labeledNameBuilder;

private MetricNameBuilder(String baseName) {
this.labeledNameBuilder = new StringBuilder(baseName + METRIC_NAME_DELIMITER);
}

public static MetricNameBuilder baseNameBuilder(String baseName) {
return new MetricNameBuilder(baseName);
}

public void addLabel(String key, String value) {
this.labeledNameBuilder
.append(key)
.append(METRIC_KV_DELIMITER)
.append(value)
.append(LABEL_DELIMITER);
}

public MetricName build(String metricNamespace) {
return MetricName.named(metricNamespace, labeledNameBuilder.toString());
}
}

/**
* Parse a 'metric name' String that was created with 'createLabeledMetricName'. The input string
* should be formatted as.
* Parse a 'metric name' String that was created with 'MetricNameBuilder'. The input string should
* be formatted as.
*
* <p>'{baseName}*{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};'
*
Expand Down Expand Up @@ -199,15 +212,14 @@ public static Optional<ParsedMetricName> parseMetricName(String metricName) {
*/
@VisibleForTesting
static Counter createRPCRequestCounter(RpcMethod method, String rpcStatus, String tableId) {
NavigableMap<String, String> metricLabels = new TreeMap<String, String>();
metricLabels.put(RPC_STATUS_LABEL, rpcStatus);
metricLabels.put(RPC_METHOD, method.toString());
MetricNameBuilder nameBuilder = MetricNameBuilder.baseNameBuilder(RPC_REQUESTS);
nameBuilder.addLabel(RPC_METHOD, method.toString());
nameBuilder.addLabel(RPC_STATUS_LABEL, rpcStatus);
if (BigQuerySinkMetrics.supportMetricsDeletion) {
metricLabels.put(TABLE_ID_LABEL, tableId);
nameBuilder.addLabel(TABLE_ID_LABEL, tableId);
}

String fullMetricName = createLabeledMetricName(RPC_REQUESTS, metricLabels);
MetricName metricName = MetricName.named(METRICS_NAMESPACE, fullMetricName);
MetricName metricName = nameBuilder.build(METRICS_NAMESPACE);
return new DelegatingCounter(metricName, false, true);
}

Expand All @@ -220,10 +232,9 @@ static Counter createRPCRequestCounter(RpcMethod method, String rpcStatus, Strin
* @return Histogram with exponential buckets with a sqrt(2) growth factor.
*/
private static Histogram createRPCLatencyHistogram(RpcMethod method) {
NavigableMap<String, String> metricLabels = new TreeMap<String, String>();
metricLabels.put(RPC_METHOD, method.toString());
String fullMetricName = createLabeledMetricName(RPC_LATENCY, metricLabels);
MetricName metricName = MetricName.named(METRICS_NAMESPACE, fullMetricName);
MetricNameBuilder nameBuilder = MetricNameBuilder.baseNameBuilder(RPC_LATENCY);
nameBuilder.addLabel(RPC_METHOD, method.toString());
MetricName metricName = nameBuilder.build(METRICS_NAMESPACE);

HistogramData.BucketType buckets = HistogramData.ExponentialBuckets.of(1, 34);

Expand Down Expand Up @@ -257,15 +268,14 @@ private static void updateRpcLatencyMetric(@Nonnull Context<?> c, RpcMethod meth
*/
public static Counter appendRowsRowStatusCounter(
RowStatus rowStatus, String rpcStatus, String tableId) {
NavigableMap<String, String> metricLabels = new TreeMap<String, String>();
metricLabels.put(RPC_STATUS_LABEL, rpcStatus);
metricLabels.put(ROW_STATUS, rowStatus.toString());
MetricNameBuilder nameBuilder = MetricNameBuilder.baseNameBuilder(APPEND_ROWS_ROW_STATUS);
nameBuilder.addLabel(ROW_STATUS, rowStatus.toString());
nameBuilder.addLabel(RPC_STATUS_LABEL, rpcStatus);
if (BigQuerySinkMetrics.supportMetricsDeletion) {
metricLabels.put(TABLE_ID_LABEL, tableId);
nameBuilder.addLabel(TABLE_ID_LABEL, tableId);
}

String fullMetricName = createLabeledMetricName(APPEND_ROWS_ROW_STATUS, metricLabels);
MetricName metricName = MetricName.named(METRICS_NAMESPACE, fullMetricName);
MetricName metricName = nameBuilder.build(METRICS_NAMESPACE);
return new DelegatingCounter(metricName, false, true);
}

Expand All @@ -274,10 +284,9 @@ public static Counter appendRowsRowStatusCounter(
* @return Counter that tracks throttled time due to RPC retries.
*/
public static Counter throttledTimeCounter(RpcMethod method) {
NavigableMap<String, String> metricLabels = new TreeMap<String, String>();
metricLabels.put(RPC_METHOD, method.toString());
String fullMetricName = createLabeledMetricName(THROTTLED_TIME, metricLabels);
MetricName metricName = MetricName.named(METRICS_NAMESPACE, fullMetricName);
MetricNameBuilder nameBuilder = MetricNameBuilder.baseNameBuilder(THROTTLED_TIME);
nameBuilder.addLabel(RPC_METHOD, method.toString());
MetricName metricName = nameBuilder.build(METRICS_NAMESPACE);

return new DelegatingCounter(metricName, false, true);
}
Expand Down

0 comments on commit ed0dc2e

Please sign in to comment.