Skip to content

Commit

Permalink
Add support for bounded trie metric in legacy worker (#33474)
Browse files Browse the repository at this point in the history
  • Loading branch information
rohitsinha54 authored Feb 6, 2025
1 parent 1ae6fc4 commit 222ad95
Show file tree
Hide file tree
Showing 13 changed files with 476 additions and 23 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,17 @@ public synchronized BoundedTrieData getCumulative() {
return value.getCumulative();
}

// Used by Streaming metric container to extract deltas since streaming metrics are
// reported as deltas rather than cumulative as in batch.
// For delta we take the current value then reset the cell to empty so the next call only see
// delta/updates from last call.
public synchronized BoundedTrieData getAndReset() {
// since we are resetting no need to do a deep copy, just change the reference
BoundedTrieData shallowCopy = this.value;
this.value = new BoundedTrieData(); // create new object, should not call reset on existing
return shallowCopy;
}

@Override
public MetricName getName() {
return name;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -180,14 +180,19 @@ public synchronized BoundedTrieResult extractResult() {
*/
public synchronized void add(Iterable<String> segments) {
List<String> segmentsParts = ImmutableList.copyOf(segments);
if (this.root == null) {
if (this.singleton == null || !this.singleton.equals(segmentsParts)) {
if (segmentsParts.isEmpty()) {
return;
}
if (this.singleton == null && this.root == null) {
// empty case
this.singleton = segmentsParts;
} else if (this.singleton != null && this.singleton.equals(segmentsParts)) {
// skip
} else {
if (this.root == null) {
this.root = this.asTrie();
this.singleton = null;
}
}

if (this.root != null) {
this.root.add(segmentsParts);
if (this.root.getSize() > this.bound) {
this.root.trim();
Expand Down Expand Up @@ -271,6 +276,11 @@ public synchronized boolean contains(@Nonnull List<String> value) {
}
}

/** @return true if this {@link BoundedTrieData} is empty else false. */
public boolean isEmpty() {
return (root == null || root.children.isEmpty()) && (singleton == null || singleton.isEmpty());
}

@Override
public final boolean equals(@Nullable Object other) {
if (this == other) {
Expand Down Expand Up @@ -335,7 +345,7 @@ static class BoundedTrieNode implements Serializable {
* @param truncated Whether this node is truncated.
* @param size The size of the subtree rooted at this node.
*/
BoundedTrieNode(Map<String, BoundedTrieNode> children, boolean truncated, int size) {
BoundedTrieNode(@Nonnull Map<String, BoundedTrieNode> children, boolean truncated, int size) {
this.children = children;
this.size = size;
this.truncated = truncated;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -882,12 +882,31 @@ public void testClear() {
assertTrue(trie.extractResult().getResult().isEmpty());
}

@Test
public void testIsEmpty() {
BoundedTrieData trie = new BoundedTrieData();
assertTrue(trie.isEmpty());

trie.add(Collections.emptyList());
assertTrue(trie.isEmpty());

trie.add(ImmutableList.of("a", "b"));
assertFalse(trie.isEmpty());

trie.add(ImmutableList.of("c", "d"));
assertFalse(trie.isEmpty());

trie.clear();
assertTrue(trie.isEmpty());
}

@Test
public void testBoundedTrieDataContains() {
BoundedTrieData trie = new BoundedTrieData();
trie.add(ImmutableList.of("a", "b"));
assertTrue(trie.contains(ImmutableList.of("a", "b")));
assertTrue(trie.contains(ImmutableList.of("a")));
// path ab is not same as path a
assertFalse(trie.contains(ImmutableList.of("a")));
assertFalse(trie.contains(ImmutableList.of("a", "c")));
}

Expand Down Expand Up @@ -1000,18 +1019,11 @@ public void testGetCumulativeWithRoot() {
assertFalse(cumulativeTrie.contains(ImmutableList.of("g", "h")));
}

@Test
public void testAddEmptyPath() {
BoundedTrieData trie = new BoundedTrieData();
trie.add(Collections.emptyList());
assertEquals(1, trie.size());
assertTrue(trie.extractResult().getResult().contains(ImmutableList.of("false")));
}

@Test
public void testContainsEmptyPath() {
BoundedTrieData trie = new BoundedTrieData();
trie.add(Collections.emptyList());
assertTrue(trie.contains(Collections.emptyList()));
assertFalse(trie.contains(Collections.emptyList()));
assertTrue(trie.isEmpty());
}
}
1 change: 1 addition & 0 deletions runners/google-cloud-dataflow-java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,7 @@ dependencies {
// io-kafka is only used in PTransform override so it is optional
provided project(":sdks:java:io:kafka")
implementation project(":sdks:java:io:google-cloud-platform")
implementation project(":runners:core-java")
implementation library.java.avro
implementation library.java.bigdataoss_util
implementation library.java.commons_codec
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,9 @@
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrie;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.runners.core.metrics.BoundedTrieData;
import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.GaugeResult;
Expand All @@ -55,6 +57,7 @@
"rawtypes", // TODO(https://github.com/apache/beam/issues/20447)
})
class DataflowMetrics extends MetricResults {

private static final Logger LOG = LoggerFactory.getLogger(DataflowMetrics.class);
/**
* Client for the Dataflow service. This can be used to query the service for information about
Expand Down Expand Up @@ -104,13 +107,13 @@ public MetricQueryResults queryMetrics(MetricsFilter filter) {
ImmutableList<MetricResult<DistributionResult>> distributions = ImmutableList.of();
ImmutableList<MetricResult<GaugeResult>> gauges = ImmutableList.of();
ImmutableList<MetricResult<StringSetResult>> stringSets = ImmutableList.of();
ImmutableList<MetricResult<BoundedTrieResult>> boudedTries = ImmutableList.of();
ImmutableList<MetricResult<BoundedTrieResult>> boundedTries = ImmutableList.of();
JobMetrics jobMetrics;
try {
jobMetrics = getJobMetrics();
} catch (IOException e) {
LOG.warn("Unable to query job metrics.\n");
return MetricQueryResults.create(counters, distributions, gauges, stringSets, boudedTries);
return MetricQueryResults.create(counters, distributions, gauges, stringSets, boundedTries);
}
metricUpdates = firstNonNull(jobMetrics.getMetrics(), Collections.emptyList());
return populateMetricQueryResults(metricUpdates, filter);
Expand All @@ -134,13 +137,15 @@ private static class DataflowMetricResultExtractor {
private final ImmutableList.Builder<MetricResult<DistributionResult>> distributionResults;
private final ImmutableList.Builder<MetricResult<GaugeResult>> gaugeResults;
private final ImmutableList.Builder<MetricResult<StringSetResult>> stringSetResults;
private final ImmutableList.Builder<MetricResult<BoundedTrieResult>> boundedTrieResults;
private final boolean isStreamingJob;

DataflowMetricResultExtractor(boolean isStreamingJob) {
counterResults = ImmutableList.builder();
distributionResults = ImmutableList.builder();
gaugeResults = ImmutableList.builder();
stringSetResults = ImmutableList.builder();
boundedTrieResults = ImmutableList.builder();
/* In Dataflow streaming jobs, only ATTEMPTED metrics are available.
* In Dataflow batch jobs, only COMMITTED metrics are available, but
* we must provide ATTEMPTED, so we use COMMITTED as a good approximation.
Expand Down Expand Up @@ -169,6 +174,9 @@ public void addMetricResult(
// stringset metric
StringSetResult value = getStringSetValue(committed);
stringSetResults.add(MetricResult.create(metricKey, !isStreamingJob, value));
} else if (committed.getTrie() != null && attempted.getTrie() != null) {
BoundedTrieResult value = getBoundedTrieValue(committed);
boundedTrieResults.add(MetricResult.create(metricKey, !isStreamingJob, value));
} else {
// This is exceptionally unexpected. We expect matching user metrics to only have the
// value types provided by the Metrics API.
Expand Down Expand Up @@ -196,6 +204,15 @@ private StringSetResult getStringSetValue(MetricUpdate metricUpdate) {
return StringSetResult.create(ImmutableSet.copyOf(((Collection) metricUpdate.getSet())));
}

private BoundedTrieResult getBoundedTrieValue(MetricUpdate metricUpdate) {
if (metricUpdate.getTrie() == null) {
return BoundedTrieResult.empty();
}
BoundedTrie bTrie = (BoundedTrie) metricUpdate.getTrie();
BoundedTrieData trieData = BoundedTrieData.fromProto(bTrie);
return BoundedTrieResult.create(trieData.extractResult().getResult());
}

private DistributionResult getDistributionValue(MetricUpdate metricUpdate) {
if (metricUpdate.getDistribution() == null) {
return DistributionResult.IDENTITY_ELEMENT;
Expand All @@ -220,9 +237,13 @@ public Iterable<MetricResult<GaugeResult>> getGaugeResults() {
return gaugeResults.build();
}

public Iterable<MetricResult<StringSetResult>> geStringSetResults() {
public Iterable<MetricResult<StringSetResult>> getStringSetResults() {
return stringSetResults.build();
}

public Iterable<MetricResult<BoundedTrieResult>> getBoundedTrieResults() {
return boundedTrieResults.build();
}
}

private static class DataflowMetricQueryResultsFactory {
Expand Down Expand Up @@ -388,8 +409,8 @@ public MetricQueryResults build() {
extractor.getCounterResults(),
extractor.getDistributionResults(),
extractor.getGaugeResults(),
extractor.geStringSetResults(),
ImmutableList.of());
extractor.getStringSetResults(),
extractor.getBoundedTrieResults());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -41,11 +41,14 @@
import java.io.IOException;
import java.math.BigDecimal;
import java.util.Set;
import org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrie;
import org.apache.beam.runners.core.metrics.BoundedTrieData;
import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
import org.apache.beam.runners.dataflow.util.DataflowTemplateJob;
import org.apache.beam.sdk.PipelineResult.State;
import org.apache.beam.sdk.extensions.gcp.auth.TestCredential;
import org.apache.beam.sdk.extensions.gcp.storage.NoopPathValidator;
import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.MetricQueryResults;
import org.apache.beam.sdk.metrics.MetricsFilter;
Expand Down Expand Up @@ -196,6 +199,13 @@ private MetricUpdate makeStringSetMetricUpdate(
return setStructuredName(update, name, namespace, step, tentative);
}

private MetricUpdate makeBoundedTrieMetricUpdate(
String name, String namespace, String step, BoundedTrie data, boolean tentative) {
MetricUpdate update = new MetricUpdate();
update.setTrie(data);
return setStructuredName(update, name, namespace, step, tentative);
}

@Test
public void testSingleCounterUpdates() throws IOException {
AppliedPTransform<?, ?, ?> myStep = mock(AppliedPTransform.class);
Expand Down Expand Up @@ -286,6 +296,64 @@ public void testSingleStringSetUpdates() throws IOException {
StringSetResult.create(ImmutableSet.of("ab", "cd")))));
}

@Test
public void testSingleBoundedTrieUpdates() throws IOException {
AppliedPTransform<?, ?, ?> myStep = mock(AppliedPTransform.class);
when(myStep.getFullName()).thenReturn("myStepName");
BiMap<AppliedPTransform<?, ?, ?>, String> transformStepNames = HashBiMap.create();
transformStepNames.put(myStep, "s2");

JobMetrics jobMetrics = new JobMetrics();
DataflowPipelineJob job = mock(DataflowPipelineJob.class);
DataflowPipelineOptions options = mock(DataflowPipelineOptions.class);
when(options.isStreaming()).thenReturn(false);
when(job.getDataflowOptions()).thenReturn(options);
when(job.getState()).thenReturn(State.RUNNING);
when(job.getJobId()).thenReturn(JOB_ID);
when(job.getTransformStepNames()).thenReturn(transformStepNames);

// The parser relies on the fact that one tentative and one committed metric update exist in
// the job metrics results.
MetricUpdate mu1 =
makeBoundedTrieMetricUpdate(
"counterName",
"counterNamespace",
"s2",
new BoundedTrieData(ImmutableList.of("ab", "cd")).toProto(),
false);
MetricUpdate mu1Tentative =
makeBoundedTrieMetricUpdate(
"counterName",
"counterNamespace",
"s2",
new BoundedTrieData(ImmutableList.of("ab", "cd")).toProto(),
true);
jobMetrics.setMetrics(ImmutableList.of(mu1, mu1Tentative));
DataflowClient dataflowClient = mock(DataflowClient.class);
when(dataflowClient.getJobMetrics(JOB_ID)).thenReturn(jobMetrics);

DataflowMetrics dataflowMetrics = new DataflowMetrics(job, dataflowClient);
MetricQueryResults result = dataflowMetrics.allMetrics();
assertThat(
result.getBoundedTries(),
containsInAnyOrder(
attemptedMetricsResult(
"counterNamespace",
"counterName",
"myStepName",
BoundedTrieResult.create(
ImmutableSet.of(ImmutableList.of("ab", "cd", String.valueOf(false)))))));
assertThat(
result.getBoundedTries(),
containsInAnyOrder(
committedMetricsResult(
"counterNamespace",
"counterName",
"myStepName",
BoundedTrieResult.create(
ImmutableSet.of(ImmutableList.of("ab", "cd", String.valueOf(false)))))));
}

@Test
public void testIgnoreDistributionButGetCounterUpdates() throws IOException {
AppliedPTransform<?, ?, ?> myStep = mock(AppliedPTransform.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -519,6 +519,11 @@ public Iterable<CounterUpdate> extractMetricUpdates(boolean isFinalUpdate) {
.transform(
update ->
MetricsToCounterUpdateConverter.fromStringSet(
update.getKey(), true, update.getUpdate())),
FluentIterable.from(updates.boundedTrieUpdates())
.transform(
update ->
MetricsToCounterUpdateConverter.fromBoundedTrie(
update.getKey(), true, update.getUpdate())));
});
}
Expand Down
Loading

0 comments on commit 222ad95

Please sign in to comment.