Skip to content

Commit

Permalink
Extract logic to build labeled MetricNames to a util Class
Browse files Browse the repository at this point in the history
  • Loading branch information
JayajP committed Apr 8, 2024
1 parent e894d8c commit 548dcda
Show file tree
Hide file tree
Showing 5 changed files with 217 additions and 157 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand All @@ -52,7 +53,7 @@ private static Optional<MetricValue> convertCounterToMetricValue(
return Optional.empty();
}

return BigQuerySinkMetrics.parseMetricName(metricName.getName())
return LabeledMetricNameUtils.parseMetricName(metricName.getName())
.filter(labeledName -> !labeledName.getBaseName().isEmpty())
.map(
labeledName ->
Expand Down Expand Up @@ -104,8 +105,8 @@ private static Optional<MetricValue> convertHistogramToMetricValue(
return Optional.empty();
}

Optional<BigQuerySinkMetrics.ParsedMetricName> labeledName =
BigQuerySinkMetrics.parseMetricName(metricName.getName());
Optional<LabeledMetricNameUtils.ParsedMetricName> labeledName =
LabeledMetricNameUtils.parseMetricName(metricName.getName());
if (!labeledName.isPresent() || labeledName.get().getBaseName().isEmpty()) {
return Optional.empty();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
/*
* 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. Avoid using labels that can be used in BigQuery Table Ids.
// 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}. Returns a metric name that merges the baseName
* with metricLables formatted as.
*
* <p>'{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);
}

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<String, String> getMetricLabels();

public static ParsedMetricName create(String baseName, Map<String, String> metricLabels) {
return new AutoValue_LabeledMetricNameUtils_ParsedMetricName(baseName, metricLabels);
}

public static ParsedMetricName create(String baseName) {
ImmutableMap<String, String> 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.
*
* <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 an empty value.
*/
public static Optional<ParsedMetricName> parseMetricName(String metricName) {
if (metricName.isEmpty()) {
return Optional.empty();
}

List<String> metricNameSplit =
Splitter.on(METRIC_NAME_DELIMITER).limit(2).splitToList(metricName);

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.
Splitter.MapSplitter splitter =
Splitter.on(LABEL_DELIMITER).omitEmptyStrings().withKeyValueSeparator(METRIC_KV_DELIMITER);
try {
Map<String, String> 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)));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* 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.ParsedMetricName expectedName =
LabeledMetricNameUtils.ParsedMetricName.create(baseMetricName);

Optional<LabeledMetricNameUtils.ParsedMetricName> parsedMetricName =
LabeledMetricNameUtils.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<String, String> metricLabels =
ImmutableMap.of("key1", "val1", "key2", "val2", "key3", "val3");
LabeledMetricNameUtils.ParsedMetricName expectedName =
LabeledMetricNameUtils.ParsedMetricName.create("baseLabel", metricLabels);

Optional<LabeledMetricNameUtils.ParsedMetricName> parsedMetricName =
LabeledMetricNameUtils.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;";
LabeledMetricNameUtils.ParsedMetricName expectedName =
LabeledMetricNameUtils.ParsedMetricName.create("baseLabel");

Optional<LabeledMetricNameUtils.ParsedMetricName> 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));
}
}

0 comments on commit 548dcda

Please sign in to comment.