Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Move logic to build a Labeled Metric Name to a util file. #30796

Merged
merged 5 commits into from
Apr 15, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
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,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:
*
* <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);
}

/**
* 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<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)));
}

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) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we log something here?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

or maybe just move the return statement below into here to show that we only do the latter when an an exception is thrown

}
return Optional.of(ParsedMetricName.create(metricNameSplit.get(0)));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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<LabeledMetricNameUtils.ParsedMetricName> 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<LabeledMetricNameUtils.ParsedMetricName> parsedName =
LabeledMetricNameUtils.parseMetricName(metricName);

String expectedMetricName = "baseMetricName*key1:val1;key2:val2;key3:val3;";
ImmutableMap<String, String> 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<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));
}
}