This is an automated email from the ASF dual-hosted git repository.
yhu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new 30917ff7b3e Don't use '-' as a delimiter in BigQuerySinkMetrics
(#30118)
30917ff7b3e is described below
commit 30917ff7b3e1009578fddd1b4d0b6b1db4776d13
Author: JayajP <[email protected]>
AuthorDate: Wed Jan 31 14:21:00 2024 -0800
Don't use '-' as a delimiter in BigQuerySinkMetrics (#30118)
* Don't use '-' as a delimiter in BigQuerySinkMetrics
* Use Optional::{filter|map} for returning simple optional
---
.../MetricsToPerStepNamespaceMetricsConverter.java | 27 +++++-----
...ricsToPerStepNamespaceMetricsConverterTest.java | 16 +++---
.../worker/StreamingStepMetricsContainerTest.java | 4 +-
.../sdk/io/gcp/bigquery/BigQuerySinkMetrics.java | 37 ++++++++------
.../io/gcp/bigquery/BigQuerySinkMetricsTest.java | 59 ++++++++++++----------
5 files changed, 75 insertions(+), 68 deletions(-)
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 f3f31ccfd36..7b72d650778 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
@@ -52,17 +52,14 @@ public class MetricsToPerStepNamespaceMetricsConverter {
return Optional.empty();
}
- BigQuerySinkMetrics.ParsedMetricName labeledName =
- BigQuerySinkMetrics.parseMetricName(metricName.getName());
- if (labeledName == null || labeledName.getBaseName().isEmpty()) {
- return Optional.empty();
- }
-
- return Optional.of(
- new MetricValue()
- .setMetric(labeledName.getBaseName())
- .setMetricLabels(labeledName.getMetricLabels())
- .setValueInt64(value));
+ return BigQuerySinkMetrics.parseMetricName(metricName.getName())
+ .filter(labeledName -> !labeledName.getBaseName().isEmpty())
+ .map(
+ labeledName ->
+ new MetricValue()
+ .setMetric(labeledName.getBaseName())
+ .setMetricLabels(labeledName.getMetricLabels())
+ .setValueInt64(value));
}
/**
@@ -78,9 +75,9 @@ public class MetricsToPerStepNamespaceMetricsConverter {
return Optional.empty();
}
- BigQuerySinkMetrics.ParsedMetricName labeledName =
+ Optional<BigQuerySinkMetrics.ParsedMetricName> labeledName =
BigQuerySinkMetrics.parseMetricName(metricName.getName());
- if (labeledName == null || labeledName.getBaseName().isEmpty()) {
+ if (!labeledName.isPresent() || labeledName.get().getBaseName().isEmpty())
{
return Optional.empty();
}
@@ -133,8 +130,8 @@ public class MetricsToPerStepNamespaceMetricsConverter {
return Optional.of(
new MetricValue()
- .setMetric(labeledName.getBaseName())
- .setMetricLabels(labeledName.getMetricLabels())
+ .setMetric(labeledName.get().getBaseName())
+ .setMetricLabels(labeledName.get().getMetricLabels())
.setValueHistogram(histogramValue));
}
diff --git
a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java
b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java
index a37bf643b7e..0a4cd06cf8c 100644
---
a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java
+++
b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java
@@ -81,7 +81,7 @@ public class MetricsToPerStepNamespaceMetricsConverterTest {
Map<MetricName, Long> counters = new HashMap<MetricName, Long>();
MetricName bigQueryMetric1 = MetricName.named("BigQuerySink", "metric1");
MetricName bigQueryMetric2 =
- MetricName.named("BigQuerySink", "metric2-label1:val1;label2:val2;");
+ MetricName.named("BigQuerySink", "metric2*label1:val1;label2:val2;");
MetricName bigQueryMetric3 = MetricName.named("BigQuerySink", "zeroValue");
counters.put(bigQueryMetric1, 5L);
@@ -112,14 +112,14 @@ public class
MetricsToPerStepNamespaceMetricsConverterTest {
@Test
public void testConvert_skipInvalidMetricNames() {
Map<MetricName, Long> counters = new HashMap<>();
- MetricName bigQueryMetric1 = MetricName.named("BigQuerySink",
"invalid-metric-name1");
- counters.put(bigQueryMetric1, 5L);
+ MetricName invalidName1 = MetricName.named("BigQuerySink", "**");
+ counters.put(invalidName1, 5L);
Map<MetricName, HistogramData> histograms = new HashMap<>();
- MetricName bigQueryMetric2 = MetricName.named("BigQuerySink",
"invalid-metric-name2");
+ MetricName invalidName2 = MetricName.named("BigQuerySink", "****");
HistogramData nonEmptyLinearHistogram = HistogramData.linear(0, 10, 10);
nonEmptyLinearHistogram.record(-5.0);
- histograms.put(bigQueryMetric2, nonEmptyLinearHistogram);
+ histograms.put(invalidName2, nonEmptyLinearHistogram);
Collection<PerStepNamespaceMetrics> conversionResult =
MetricsToPerStepNamespaceMetricsConverter.convert("testStep",
counters, histograms);
@@ -131,7 +131,7 @@ public class MetricsToPerStepNamespaceMetricsConverterTest {
Map<MetricName, HistogramData> histograms = new HashMap<MetricName,
HistogramData>();
MetricName bigQueryMetric1 = MetricName.named("BigQuerySink", "baseLabel");
MetricName bigQueryMetric2 =
- MetricName.named("BigQuerySink", "baseLabel-label1:val1;label2:val2;");
+ MetricName.named("BigQuerySink", "baseLabel*label1:val1;label2:val2;");
MetricName bigQueryMetric3 = MetricName.named("BigQuerySink", "zeroValue");
HistogramData nonEmptyLinearHistogram = HistogramData.linear(0, 10, 10);
@@ -235,10 +235,10 @@ public class
MetricsToPerStepNamespaceMetricsConverterTest {
Map<MetricName, Long> counters = new HashMap<>();
Map<MetricName, HistogramData> histograms = new HashMap<MetricName,
HistogramData>();
- MetricName counterMetricName = MetricName.named("BigQuerySink",
"counter-label1:val1;");
+ MetricName counterMetricName = MetricName.named("BigQuerySink",
"counter*label1:val1;");
counters.put(counterMetricName, 3L);
- MetricName histogramMetricName = MetricName.named("BigQuerySink",
"histogram-label2:val2;");
+ MetricName histogramMetricName = MetricName.named("BigQuerySink",
"histogram*label2:val2;");
HistogramData linearHistogram = HistogramData.linear(0, 10, 10);
linearHistogram.record(5.0);
histograms.put(histogramMetricName, linearHistogram);
diff --git
a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java
b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java
index cc0722e7a8e..c586f2b8781 100644
---
a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java
+++
b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java
@@ -226,10 +226,10 @@ public class StreamingStepMetricsContainerTest {
@Test
public void testExtractPerWorkerMetricUpdates() {
StreamingStepMetricsContainer.setEnablePerWorkerMetrics(true);
- MetricName counterMetricName = MetricName.named("BigQuerySink",
"counter-");
+ MetricName counterMetricName = MetricName.named("BigQuerySink", "counter");
c1.getPerWorkerCounter(counterMetricName).inc(3);
- MetricName histogramMetricName = MetricName.named("BigQuerySink",
"histogram-");
+ MetricName histogramMetricName = MetricName.named("BigQuerySink",
"histogram");
HistogramData.LinearBuckets linearBuckets =
HistogramData.LinearBuckets.of(0, 10, 10);
c2.getPerWorkerHistogram(histogramMetricName, linearBuckets).update(5.0);
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 c0f470b3921..34e3b704b4f 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
@@ -22,6 +22,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;
@@ -40,10 +41,10 @@ import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Immuta
*
* <p>In general metrics be in the namespace 'BigQuerySink' and have their
name formatted as:
*
- *
<p>'{baseName}-{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};'
+ *
<p>'{baseName}*{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};'
*/
public class BigQuerySinkMetrics {
- private static Boolean supportMetricsDeletion = false;
+ private static boolean supportMetricsDeletion = false;
public static final String METRICS_NAMESPACE = "BigQuerySink";
@@ -78,10 +79,11 @@ public class BigQuerySinkMetrics {
private static final String RPC_METHOD = "rpc_method";
private static final String ROW_STATUS = "row_status";
- // Delimiters
+ // 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 = '-';
+ private static final char METRIC_NAME_DELIMITER = '*';
@AutoValue
public abstract static class ParsedMetricName {
@@ -119,38 +121,41 @@ public class BigQuerySinkMetrics {
* Parse a 'metric name' String that was created with
'createLabeledMetricName'. The input string
* should be formatted as.
*
- *
<p>'{baseName}-{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};'
+ *
<p>'{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 null.
+ * input string is empty or malformed, returns an empty value.
*/
- public static @Nullable ParsedMetricName parseMetricName(String metricName) {
+ public static Optional<ParsedMetricName> parseMetricName(String metricName) {
if (metricName.isEmpty()) {
- return null;
+ return Optional.empty();
}
- List<String> metricNameSplit =
Splitter.on(METRIC_NAME_DELIMITER).splitToList(metricName);
+ List<String> metricNameSplit =
+ Splitter.on(METRIC_NAME_DELIMITER).limit(2).splitToList(metricName);
ImmutableMap.Builder<String, String> metricLabelsBuilder =
ImmutableMap.builder();
- if (metricNameSplit.size() == 1) {
- return ParsedMetricName.create(metricNameSplit.get(0));
+ if (metricNameSplit.size() == 0) {
+ return Optional.empty();
}
- if (metricNameSplit.size() != 2) {
- return null;
+ if (metricNameSplit.size() == 1) {
+ return Optional.of(ParsedMetricName.create(metricNameSplit.get(0)));
}
+ // metrcNameSplit is assumed to be size two.
List<String> labels =
Splitter.on(LABEL_DELIMITER).splitToList(metricNameSplit.get(1));
for (String label : labels) {
- List<String> kv = Splitter.on(METRIC_KV_DELIMITER).splitToList(label);
+ List<String> kv =
Splitter.on(METRIC_KV_DELIMITER).limit(2).splitToList(label);
if (kv.size() != 2) {
continue;
}
metricLabelsBuilder.put(kv.get(0), kv.get(1));
}
- return ParsedMetricName.create(metricNameSplit.get(0),
metricLabelsBuilder.build());
+ return Optional.of(
+ ParsedMetricName.create(metricNameSplit.get(0),
metricLabelsBuilder.build()));
}
/**
@@ -321,7 +326,7 @@ public class BigQuerySinkMetrics {
updateRpcLatencyMetric(c, method);
}
- public static void setSupportMetricsDeletion(Boolean supportMetricsDeletion)
{
+ public static void setSupportMetricsDeletion(boolean supportMetricsDeletion)
{
BigQuerySinkMetrics.supportMetricsDeletion = supportMetricsDeletion;
}
}
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 6b04ed0acc1..8b960a8c6c6 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
@@ -20,14 +20,13 @@ package org.apache.beam.sdk.io.gcp.bigquery;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.nullValue;
import com.google.cloud.bigquery.storage.v1.AppendRowsResponse;
import com.google.cloud.bigquery.storage.v1.Exceptions;
import io.grpc.Status;
import java.time.Instant;
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;
@@ -112,7 +111,7 @@ public class BigQuerySinkMetricsTest {
deletesDisabledCounter.inc();
MetricName deletesDisabledCounterName =
MetricName.named(
- "BigQuerySink",
"RowsAppendedCount-row_status:SUCCESSFUL;rpc_status:rpcStatus;");
+ "BigQuerySink",
"RowsAppendedCount*row_status:SUCCESSFUL;rpc_status:rpcStatus;");
assertThat(testContainer.perWorkerCounters,
IsMapContaining.hasKey(deletesDisabledCounterName));
assertThat(
testContainer.perWorkerCounters.get(deletesDisabledCounterName).getCumulative(),
@@ -127,7 +126,7 @@ public class BigQuerySinkMetricsTest {
MetricName deletesEnabledCounterName =
MetricName.named(
"BigQuerySink",
-
"RowsAppendedCount-row_status:SUCCESSFUL;rpc_status:rpcStatus;table_id:tableId;");
+
"RowsAppendedCount*row_status:SUCCESSFUL;rpc_status:rpcStatus;table_id:tableId;");
assertThat(testContainer.perWorkerCounters,
IsMapContaining.hasKey(deletesEnabledCounterName));
assertThat(
testContainer.perWorkerCounters.get(deletesEnabledCounterName).getCumulative(),
@@ -160,9 +159,10 @@ public class BigQuerySinkMetricsTest {
appendRowsThrottleCounter.inc(1);
assertThat(
appendRowsThrottleCounter.getName().getName(),
- equalTo("ThrottledTime-rpc_method:APPEND_ROWS;"));
+ equalTo("ThrottledTime*rpc_method:APPEND_ROWS;"));
+
MetricName counterName =
- MetricName.named("BigQuerySink",
"ThrottledTime-rpc_method:APPEND_ROWS;");
+ MetricName.named("BigQuerySink",
"ThrottledTime*rpc_method:APPEND_ROWS;");
assertThat(testContainer.perWorkerCounters,
IsMapContaining.hasKey(counterName));
assertThat(testContainer.perWorkerCounters.get(counterName).getCumulative(),
equalTo(1L));
}
@@ -182,9 +182,9 @@ public class BigQuerySinkMetricsTest {
BigQuerySinkMetrics.reportSuccessfulRpcMetrics(
c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId");
MetricName counterNameDisabledDeletes =
- MetricName.named("BigQuerySink",
"RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:OK;");
+ MetricName.named("BigQuerySink",
"RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:OK;");
MetricName histogramName =
- MetricName.named("BigQuerySink", "RpcLatency-rpc_method:APPEND_ROWS;");
+ MetricName.named("BigQuerySink", "RpcLatency*rpc_method:APPEND_ROWS;");
HistogramData.BucketType bucketType =
HistogramData.ExponentialBuckets.of(1, 34);
assertThat(testContainer.perWorkerCounters,
IsMapContaining.hasKey(counterNameDisabledDeletes));
assertThat(
@@ -202,7 +202,7 @@ public class BigQuerySinkMetricsTest {
MetricName counterNameEnabledDeletes =
MetricName.named(
"BigQuerySink",
-
"RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:OK;table_id:tableId;");
+
"RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:OK;table_id:tableId;");
assertThat(testContainer.perWorkerCounters,
IsMapContaining.hasKey(counterNameEnabledDeletes));
assertThat(
testContainer.perWorkerCounters.get(counterNameEnabledDeletes).getCumulative(),
@@ -232,9 +232,9 @@ public class BigQuerySinkMetricsTest {
c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId");
MetricName counterNameDisabledDeletes =
MetricName.named(
- "BigQuerySink",
"RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:NOT_FOUND;");
+ "BigQuerySink",
"RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:NOT_FOUND;");
MetricName histogramName =
- MetricName.named("BigQuerySink", "RpcLatency-rpc_method:APPEND_ROWS;");
+ MetricName.named("BigQuerySink", "RpcLatency*rpc_method:APPEND_ROWS;");
HistogramData.BucketType bucketType =
HistogramData.ExponentialBuckets.of(1, 34);
assertThat(testContainer.perWorkerCounters,
IsMapContaining.hasKey(counterNameDisabledDeletes));
assertThat(
@@ -255,7 +255,7 @@ public class BigQuerySinkMetricsTest {
MetricName counterNameEnabledDeletes =
MetricName.named(
"BigQuerySink",
-
"RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:NOT_FOUND;table_id:tableId;");
+
"RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:NOT_FOUND;table_id:tableId;");
assertThat(testContainer.perWorkerCounters,
IsMapContaining.hasKey(counterNameEnabledDeletes));
assertThat(
testContainer.perWorkerCounters.get(counterNameEnabledDeletes).getCumulative(),
@@ -284,9 +284,9 @@ public class BigQuerySinkMetricsTest {
c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId");
MetricName counterNameDisabledDeletes =
MetricName.named(
- "BigQuerySink",
"RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:UNKNOWN;");
+ "BigQuerySink",
"RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:UNKNOWN;");
MetricName histogramName =
- MetricName.named("BigQuerySink", "RpcLatency-rpc_method:APPEND_ROWS;");
+ MetricName.named("BigQuerySink", "RpcLatency*rpc_method:APPEND_ROWS;");
HistogramData.BucketType bucketType =
HistogramData.ExponentialBuckets.of(1, 34);
assertThat(testContainer.perWorkerCounters,
IsMapContaining.hasKey(counterNameDisabledDeletes));
assertThat(
@@ -304,7 +304,7 @@ public class BigQuerySinkMetricsTest {
MetricName counterNameEnabledDeletes =
MetricName.named(
"BigQuerySink",
-
"RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:UNKNOWN;table_id:tableId;");
+
"RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:UNKNOWN;table_id:tableId;");
assertThat(testContainer.perWorkerCounters,
IsMapContaining.hasKey(counterNameEnabledDeletes));
assertThat(
testContainer.perWorkerCounters.get(counterNameEnabledDeletes).getCumulative(),
@@ -317,41 +317,46 @@ public class BigQuerySinkMetricsTest {
@Test
public void testParseMetricName_noLabels() {
String baseMetricName = "baseMetricName";
- BigQuerySinkMetrics.ParsedMetricName metricName =
+ BigQuerySinkMetrics.ParsedMetricName expectedName =
BigQuerySinkMetrics.ParsedMetricName.create(baseMetricName);
- assertThat(BigQuerySinkMetrics.parseMetricName(baseMetricName),
equalTo(metricName));
+ Optional<BigQuerySinkMetrics.ParsedMetricName> 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;";
+ String metricName = "baseLabel*key1:val1;key2:val2;key3:val3;";
ImmutableMap<String, String> metricLabels =
ImmutableMap.of("key1", "val1", "key2", "val2", "key3", "val3");
BigQuerySinkMetrics.ParsedMetricName expectedName =
BigQuerySinkMetrics.ParsedMetricName.create("baseLabel", metricLabels);
- assertThat(BigQuerySinkMetrics.parseMetricName(metricName),
equalTo(expectedName));
- }
+ Optional<BigQuerySinkMetrics.ParsedMetricName> parsedMetricName =
+ BigQuerySinkMetrics.parseMetricName(metricName);
- @Test
- public void testParseMetricName_malformedMetricName() {
- String malformedMetricName = "baseLabel-key1:val1-key2:val2";
- assertThat(BigQuerySinkMetrics.parseMetricName(malformedMetricName),
is(nullValue()));
+ assertThat(parsedMetricName.isPresent(), equalTo(true));
+ assertThat(parsedMetricName.get(), equalTo(expectedName));
}
@Test
public void testParseMetricName_malformedMetricLabels() {
- String metricName = "baseLabel-key1:val1:malformedField;key2:val2;";
+ String metricName = "baseLabel*malformed_kv_pair;key2:val2;";
ImmutableMap<String, String> metricLabels = ImmutableMap.of("key2",
"val2");
BigQuerySinkMetrics.ParsedMetricName expectedName =
BigQuerySinkMetrics.ParsedMetricName.create("baseLabel", metricLabels);
- assertThat(BigQuerySinkMetrics.parseMetricName(metricName),
equalTo(expectedName));
+ Optional<BigQuerySinkMetrics.ParsedMetricName> parsedMetricName =
+ BigQuerySinkMetrics.parseMetricName(metricName);
+
+ assertThat(parsedMetricName.isPresent(), equalTo(true));
+ assertThat(parsedMetricName.get(), equalTo(expectedName));
}
@Test
public void testParseMetricName_emptyString() {
- assertThat(BigQuerySinkMetrics.parseMetricName(""), is(nullValue()));
+ assertThat(BigQuerySinkMetrics.parseMetricName("").isPresent(),
equalTo(false));
}
}