From 41910de15c1e1470db466922805ce6c208deb2aa Mon Sep 17 00:00:00 2001 From: JayajP Date: Mon, 15 Apr 2024 11:19:42 -0700 Subject: [PATCH] Move logic to build a Labeled Metric Name to a util file. (#30796) * Extract logic to build labeled MetricNames to a util Class * Address comments --- ...icsToPerStepNamespaceMetricsConverter.java | 7 +- .../sdk/metrics/LabeledMetricNameUtils.java | 122 +++++++++++++++++ .../metrics/LabeledMetricNameUtilsTest.java | 96 +++++++++++++ .../io/gcp/bigquery/BigQuerySinkMetrics.java | 127 +++--------------- .../gcp/bigquery/BigQuerySinkMetricsTest.java | 48 ------- 5 files changed, 243 insertions(+), 157 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/LabeledMetricNameUtils.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/LabeledMetricNameUtilsTest.java diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java index 1f54ee95dad7..05752aae24ad 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java @@ -32,6 +32,7 @@ import java.util.Map.Entry; import java.util.Optional; import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySinkMetrics; +import org.apache.beam.sdk.metrics.LabeledMetricNameUtils; import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.util.HistogramData; @@ -52,7 +53,7 @@ private static Optional convertCounterToMetricValue( return Optional.empty(); } - return BigQuerySinkMetrics.parseMetricName(metricName.getName()) + return LabeledMetricNameUtils.parseMetricName(metricName.getName()) .filter(labeledName -> !labeledName.getBaseName().isEmpty()) .map( labeledName -> @@ -104,8 +105,8 @@ private static Optional convertHistogramToMetricValue( return Optional.empty(); } - Optional labeledName = - BigQuerySinkMetrics.parseMetricName(metricName.getName()); + Optional labeledName = + LabeledMetricNameUtils.parseMetricName(metricName.getName()); if (!labeledName.isPresent() || labeledName.get().getBaseName().isEmpty()) { return Optional.empty(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/LabeledMetricNameUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/LabeledMetricNameUtils.java new file mode 100644 index 000000000000..72b303632331 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/LabeledMetricNameUtils.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.beam.sdk.metrics; + +import com.google.auto.value.AutoValue; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; + +/** Util class for building/parsing labeled {@link MetricName}. */ +public class LabeledMetricNameUtils { + // Delimiters. Avoid using delimeters that can be used in metric labels. + // E.g. Since we support BigQuery TableIds as labels we avoid using delimiters that can be used in + // TableIds. + // ref: https://cloud.google.com/bigquery/docs/tables#table_naming + private static final char LABEL_DELIMITER = ';'; + private static final char METRIC_KV_DELIMITER = ':'; + private static final char METRIC_NAME_DELIMITER = '*'; + + /** + * Builder class for a labeled {@code MetricName}. Returned {@link MetricName#getName} will be + * formatted as: + * + *

'{baseName}-{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};' + */ + public 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); + } + + /** + * Add a metric label KV pair to the metric name. Calling {@code addLabel} multiple times with + * the same {@code key} will lead to unexpected results when parsing metric names. + */ + 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()); + } + } + + @AutoValue + public abstract static class ParsedMetricName { + public abstract String getBaseName(); + + public abstract Map getMetricLabels(); + + public static ParsedMetricName create(String baseName, Map metricLabels) { + return new AutoValue_LabeledMetricNameUtils_ParsedMetricName(baseName, metricLabels); + } + + public static ParsedMetricName create(String baseName) { + ImmutableMap emptyMap = ImmutableMap.of(); + return new AutoValue_LabeledMetricNameUtils_ParsedMetricName(baseName, emptyMap); + } + } + + /** + * Parse a 'metric name' String that was created with 'MetricNameBuilder'. The input string should + * be formatted as. + * + *

'{baseName}*{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};' + * + * @param metricName + * @return Returns a ParsedMetricName object if the input string is properly formatted. If the + * input string is empty or malformed, returns an empty value. + */ + public static Optional parseMetricName(String metricName) { + if (metricName.isEmpty()) { + return Optional.empty(); + } + + List metricNameSplit = + Splitter.on(METRIC_NAME_DELIMITER).limit(2).splitToList(metricName); + + if (metricNameSplit.size() == 0 || metricNameSplit.get(0).isEmpty()) { + return Optional.empty(); + } + + if (metricNameSplit.size() == 1) { + return Optional.of(ParsedMetricName.create(metricNameSplit.get(0))); + } + + Splitter.MapSplitter splitter = + Splitter.on(LABEL_DELIMITER).omitEmptyStrings().withKeyValueSeparator(METRIC_KV_DELIMITER); + try { + Map labels = splitter.split(metricNameSplit.get(1)); + return Optional.of(ParsedMetricName.create(metricNameSplit.get(0), labels)); + } catch (IllegalArgumentException e) { + return Optional.of(ParsedMetricName.create(metricNameSplit.get(0))); + } + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/LabeledMetricNameUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/LabeledMetricNameUtilsTest.java new file mode 100644 index 000000000000..932a8a0e9029 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/LabeledMetricNameUtilsTest.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.beam.sdk.metrics; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +import java.io.Serializable; +import java.util.Optional; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.junit.Test; + +public class LabeledMetricNameUtilsTest implements Serializable { + @Test + public void testParseMetricName_noLabels() { + String baseMetricName = "baseMetricName"; + LabeledMetricNameUtils.MetricNameBuilder builder = + LabeledMetricNameUtils.MetricNameBuilder.baseNameBuilder(baseMetricName); + String metricName = builder.build("namespace").getName(); + Optional parsedName = + LabeledMetricNameUtils.parseMetricName(metricName); + + LabeledMetricNameUtils.ParsedMetricName expectedParsedName = + LabeledMetricNameUtils.ParsedMetricName.create(baseMetricName); + + assertThat(parsedName.isPresent(), equalTo(true)); + assertThat(parsedName.get(), equalTo(expectedParsedName)); + assertThat(parsedName.get().getBaseName(), equalTo(baseMetricName)); + } + + @Test + public void testParseMetricName_successfulLabels() { + String baseMetricName = "baseMetricName"; + LabeledMetricNameUtils.MetricNameBuilder builder = + LabeledMetricNameUtils.MetricNameBuilder.baseNameBuilder(baseMetricName); + builder.addLabel("key1", "val1"); + builder.addLabel("key2", "val2"); + builder.addLabel("key3", "val3"); + String metricName = builder.build("namespace").getName(); + Optional parsedName = + LabeledMetricNameUtils.parseMetricName(metricName); + + String expectedMetricName = "baseMetricName*key1:val1;key2:val2;key3:val3;"; + ImmutableMap expectedLabels = + ImmutableMap.of("key1", "val1", "key2", "val2", "key3", "val3"); + LabeledMetricNameUtils.ParsedMetricName expectedParsedName = + LabeledMetricNameUtils.ParsedMetricName.create(baseMetricName, expectedLabels); + + assertThat(metricName, equalTo(expectedMetricName)); + assertThat(parsedName.isPresent(), equalTo(true)); + assertThat(parsedName.get(), equalTo(expectedParsedName)); + assertThat(parsedName.get().getBaseName(), equalTo(baseMetricName)); + assertThat(parsedName.get().getMetricLabels(), equalTo(expectedLabels)); + } + + @Test + public void testParseMetricName_malformedMetricLabels() { + String metricName = "baseLabel*malformed_kv_pair;key2:val2;"; + LabeledMetricNameUtils.ParsedMetricName expectedName = + LabeledMetricNameUtils.ParsedMetricName.create("baseLabel"); + + Optional parsedMetricName = + LabeledMetricNameUtils.parseMetricName(metricName); + + assertThat(parsedMetricName.isPresent(), equalTo(true)); + assertThat(parsedMetricName.get(), equalTo(expectedName)); + } + + @Test + public void testParseMetricName_emptyString() { + assertThat(LabeledMetricNameUtils.parseMetricName("").isPresent(), equalTo(false)); + } + + @Test + public void testParseMetricName_emptyMetric() { + LabeledMetricNameUtils.MetricNameBuilder builder = + LabeledMetricNameUtils.MetricNameBuilder.baseNameBuilder(""); + String metricName = builder.build("namespace").getName(); + assertThat(LabeledMetricNameUtils.parseMetricName(metricName).isPresent(), equalTo(false)); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java index 392a1d164047..b80438aceeb8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java @@ -17,13 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -import com.google.auto.value.AutoValue; 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; @@ -31,11 +26,10 @@ import org.apache.beam.sdk.metrics.DelegatingCounter; import org.apache.beam.sdk.metrics.DelegatingHistogram; import org.apache.beam.sdk.metrics.Histogram; +import org.apache.beam.sdk.metrics.LabeledMetricNameUtils; import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.util.HistogramData; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** * Helper class to create perworker metrics for BigQuery Sink stages. @@ -83,85 +77,6 @@ enum RowStatus { private static final String RPC_METHOD = "rpc_method"; private static final String ROW_STATUS = "row_status"; - // Delimiters. Avoid using dilimiters that can also be used in a BigQuery table name. - // ref: https://cloud.google.com/bigquery/docs/tables#table_naming - private static final char LABEL_DELIMITER = ';'; - private static final char METRIC_KV_DELIMITER = ':'; - private static final char METRIC_NAME_DELIMITER = '*'; - - @AutoValue - public abstract static class ParsedMetricName { - public abstract String getBaseName(); - - public abstract ImmutableMap getMetricLabels(); - - public static ParsedMetricName create( - String baseName, ImmutableMap metricLabels) { - return new AutoValue_BigQuerySinkMetrics_ParsedMetricName(baseName, metricLabels); - } - - public static ParsedMetricName create(String baseName) { - ImmutableMap emptyMap = ImmutableMap.of(); - return new AutoValue_BigQuerySinkMetrics_ParsedMetricName(baseName, emptyMap); - } - } - - /** - * Returns a metric name that merges the baseName with metricLables formatted as. - * - *

'{baseName}-{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};' - */ - private static String createLabeledMetricName( - String baseName, NavigableMap 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(); - } - - /** - * Parse a 'metric name' String that was created with 'createLabeledMetricName'. The input string - * should be formatted as. - * - *

'{baseName}*{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};' - * - * @param metricName - * @return Returns a ParsedMetricName object if the input string is properly formatted. If the - * input string is empty or malformed, returns an empty value. - */ - public static Optional parseMetricName(String metricName) { - if (metricName.isEmpty()) { - return Optional.empty(); - } - - List metricNameSplit = - Splitter.on(METRIC_NAME_DELIMITER).limit(2).splitToList(metricName); - ImmutableMap.Builder metricLabelsBuilder = ImmutableMap.builder(); - - if (metricNameSplit.size() == 0) { - return Optional.empty(); - } - - if (metricNameSplit.size() == 1) { - return Optional.of(ParsedMetricName.create(metricNameSplit.get(0))); - } - // metrcNameSplit is assumed to be size two. - - List labels = Splitter.on(LABEL_DELIMITER).splitToList(metricNameSplit.get(1)); - for (String label : labels) { - List kv = Splitter.on(METRIC_KV_DELIMITER).limit(2).splitToList(label); - if (kv.size() != 2) { - continue; - } - metricLabelsBuilder.put(kv.get(0), kv.get(1)); - } - - return Optional.of( - ParsedMetricName.create(metricNameSplit.get(0), metricLabelsBuilder.build())); - } - /** * @param method StorageWriteAPI method associated with this metric. * @param rpcStatus RPC return status. @@ -173,15 +88,15 @@ public static Optional parseMetricName(String metricName) { */ @VisibleForTesting static Counter createRPCRequestCounter(RpcMethod method, String rpcStatus, String tableId) { - NavigableMap metricLabels = new TreeMap(); - metricLabels.put(RPC_STATUS_LABEL, rpcStatus); - metricLabels.put(RPC_METHOD, method.toString()); + LabeledMetricNameUtils.MetricNameBuilder nameBuilder = + LabeledMetricNameUtils.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); } @@ -194,10 +109,10 @@ static Counter createRPCRequestCounter(RpcMethod method, String rpcStatus, Strin * @return Histogram with exponential buckets with a sqrt(2) growth factor. */ static Histogram createRPCLatencyHistogram(RpcMethod method) { - NavigableMap metricLabels = new TreeMap(); - metricLabels.put(RPC_METHOD, method.toString()); - String fullMetricName = createLabeledMetricName(RPC_LATENCY, metricLabels); - MetricName metricName = MetricName.named(METRICS_NAMESPACE, fullMetricName); + LabeledMetricNameUtils.MetricNameBuilder nameBuilder = + LabeledMetricNameUtils.MetricNameBuilder.baseNameBuilder(RPC_LATENCY); + nameBuilder.addLabel(RPC_METHOD, method.toString()); + MetricName metricName = nameBuilder.build(METRICS_NAMESPACE); HistogramData.BucketType buckets = HistogramData.ExponentialBuckets.of(1, 34); @@ -231,15 +146,15 @@ private static void updateRpcLatencyMetric(@Nonnull Context c, RpcMethod meth */ public static Counter appendRowsRowStatusCounter( RowStatus rowStatus, String rpcStatus, String tableId) { - NavigableMap metricLabels = new TreeMap(); - metricLabels.put(RPC_STATUS_LABEL, rpcStatus); - metricLabels.put(ROW_STATUS, rowStatus.toString()); + LabeledMetricNameUtils.MetricNameBuilder nameBuilder = + LabeledMetricNameUtils.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); } @@ -248,10 +163,10 @@ public static Counter appendRowsRowStatusCounter( * @return Counter that tracks throttled time due to RPC retries. */ public static Counter throttledTimeCounter(RpcMethod method) { - NavigableMap metricLabels = new TreeMap(); - metricLabels.put(RPC_METHOD, method.toString()); - String fullMetricName = createLabeledMetricName(THROTTLED_TIME, metricLabels); - MetricName metricName = MetricName.named(METRICS_NAMESPACE, fullMetricName); + LabeledMetricNameUtils.MetricNameBuilder nameBuilder = + LabeledMetricNameUtils.MetricNameBuilder.baseNameBuilder(THROTTLED_TIME); + nameBuilder.addLabel(RPC_METHOD, method.toString()); + MetricName metricName = nameBuilder.build(METRICS_NAMESPACE); return new DelegatingCounter(metricName, false, true); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java index 861378a41f8e..37472c4db3df 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java @@ -29,7 +29,6 @@ import java.time.Instant; import java.util.Arrays; import java.util.List; -import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import org.apache.beam.runners.core.metrics.CounterCell; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; @@ -40,7 +39,6 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.util.HistogramData; import org.apache.beam.sdk.values.KV; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.hamcrest.collection.IsMapContaining; import org.junit.Test; @@ -332,52 +330,6 @@ public void testReportFailedRPCMetrics_UnknownGrpcError() throws Exception { containsInAnyOrder(Double.valueOf(15.0))); } - @Test - public void testParseMetricName_noLabels() { - String baseMetricName = "baseMetricName"; - BigQuerySinkMetrics.ParsedMetricName expectedName = - BigQuerySinkMetrics.ParsedMetricName.create(baseMetricName); - - Optional parsedMetricName = - BigQuerySinkMetrics.parseMetricName(baseMetricName); - assertThat(parsedMetricName.isPresent(), equalTo(true)); - assertThat(parsedMetricName.get(), equalTo(expectedName)); - } - - @Test - public void testParseMetricName_successfulLabels() { - String metricName = "baseLabel*key1:val1;key2:val2;key3:val3;"; - ImmutableMap metricLabels = - ImmutableMap.of("key1", "val1", "key2", "val2", "key3", "val3"); - BigQuerySinkMetrics.ParsedMetricName expectedName = - BigQuerySinkMetrics.ParsedMetricName.create("baseLabel", metricLabels); - - Optional parsedMetricName = - BigQuerySinkMetrics.parseMetricName(metricName); - - assertThat(parsedMetricName.isPresent(), equalTo(true)); - assertThat(parsedMetricName.get(), equalTo(expectedName)); - } - - @Test - public void testParseMetricName_malformedMetricLabels() { - String metricName = "baseLabel*malformed_kv_pair;key2:val2;"; - ImmutableMap metricLabels = ImmutableMap.of("key2", "val2"); - BigQuerySinkMetrics.ParsedMetricName expectedName = - BigQuerySinkMetrics.ParsedMetricName.create("baseLabel", metricLabels); - - Optional parsedMetricName = - BigQuerySinkMetrics.parseMetricName(metricName); - - assertThat(parsedMetricName.isPresent(), equalTo(true)); - assertThat(parsedMetricName.get(), equalTo(expectedName)); - } - - @Test - public void testParseMetricName_emptyString() { - assertThat(BigQuerySinkMetrics.parseMetricName("").isPresent(), equalTo(false)); - } - @Test public void testStreamingInsertsMetrics_disabled() { BigQuerySinkMetrics.setSupportStreamingInsertsMetrics(false);