diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/BoundedTrieCell.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/BoundedTrieCell.java
new file mode 100644
index 000000000000..e749fed73c2c
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/BoundedTrieCell.java
@@ -0,0 +1,104 @@
+/*
+ * 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.runners.core.metrics;
+
+import java.util.Arrays;
+import java.util.Objects;
+import org.apache.beam.sdk.metrics.BoundedTrie;
+import org.apache.beam.sdk.metrics.MetricName;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Tracks the current value for a {@link BoundedTrie} metric.
+ *
+ *
This class generally shouldn't be used directly. The only exception is within a runner where a
+ * counter is being reported for a specific step (rather than the counter in the current context).
+ * In that case retrieving the underlying cell and reporting directly to it avoids a step of
+ * indirection.
+ */
+public class BoundedTrieCell implements BoundedTrie, MetricCell {
+
+ private final DirtyState dirty = new DirtyState();
+ private BoundedTrieData value;
+ private final MetricName name;
+
+ public BoundedTrieCell(MetricName name) {
+ this.name = name;
+ this.value = new BoundedTrieData();
+ }
+
+ public synchronized void update(BoundedTrieData other) {
+ // although BoundedTrieData is thread-safe the cell is made thread safe too because combine
+ // returns a reference to the combined BoundedTrieData and want the reference update here to
+ // be thread safe too.
+ this.value = this.value.combine(other);
+ dirty.afterModification();
+ }
+
+ @Override
+ public synchronized void reset() {
+ value.clear();
+ dirty.reset();
+ }
+
+ @Override
+ public DirtyState getDirty() {
+ return dirty;
+ }
+
+ /**
+ * @return Returns a deep copy of the {@link BoundedTrieData} contained in this {@link
+ * BoundedTrieCell}.
+ */
+ @Override
+ public synchronized BoundedTrieData getCumulative() {
+ return value.getCumulative();
+ }
+
+ @Override
+ public MetricName getName() {
+ return name;
+ }
+
+ @Override
+ public boolean equals(@Nullable Object object) {
+ if (object instanceof BoundedTrieCell) {
+ BoundedTrieCell boundedTrieCell = (BoundedTrieCell) object;
+ return Objects.equals(dirty, boundedTrieCell.dirty)
+ && Objects.equals(value, boundedTrieCell.value)
+ && Objects.equals(name, boundedTrieCell.name);
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(dirty, value, name);
+ }
+
+ @Override
+ public synchronized void add(Iterable values) {
+ this.value.add(values);
+ dirty.afterModification();
+ }
+
+ @Override
+ public synchronized void add(String... values) {
+ add(Arrays.asList(values));
+ }
+}
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/BoundedTrieData.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/BoundedTrieData.java
new file mode 100644
index 000000000000..049ba5c93967
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/BoundedTrieData.java
@@ -0,0 +1,594 @@
+/*
+ * 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.runners.core.metrics;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrie;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
+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.collect.ImmutableList;
+
+/**
+ * Experimental and subject to incompatible changes, or even removal, in a future releases.
+ *
+ * Represents data stored in a bounded trie. This data structure is used to efficiently store and
+ * aggregate a collection of string sequences, paths/FQN with a limited size.
+ *
+ *
This class is thread-safe but the underlying BoundedTrieNode contained in it isn't. This is
+ * intentional for performance concerns. Hence, this class does not expose the contained node and
+ * should not be modified to do so in future when used with multiple threads. This class choose to
+ * achieve thread-safety through locks rather than just creating and returning immutable instances
+ * to its caller because the combining of a large and wide trie require per-node copy which is more
+ * expensive than synchronization.
+ *
+ *
Note: {@link #equals(Object)}, {@link #hashCode()} of this class are not synchronized and if
+ * their usage needs synchronization then the client should do it.
+ */
+@Internal
+@SuppressWarnings({
+ "nullness" // TODO(https://github.com/apache/beam/issues/20497)
+})
+public class BoundedTrieData implements Serializable {
+
+ private static final int DEFAULT_BOUND = 100; // Default maximum size of the trie
+
+ /** Returns the singleton path if this {@link BoundedTrieData} represents a single path. */
+ @Nullable private List singleton;
+
+ /**
+ * Returns an {@link Optional} containing the root {@link BoundedTrieNode} if this {@link
+ * BoundedTrieData} represents a trie.
+ */
+ @Nullable private BoundedTrieNode root;
+
+ /** Returns the maximum size of the trie. */
+ private int bound;
+
+ public BoundedTrieData() {
+ this(null, null, DEFAULT_BOUND);
+ }
+
+ public BoundedTrieData(List singleton) {
+ this(singleton, null, DEFAULT_BOUND);
+ }
+
+ public BoundedTrieData(BoundedTrieNode root) {
+ this(null, root, DEFAULT_BOUND);
+ }
+
+ /**
+ * Constructs a new BoundedTrieData object.
+ *
+ * A BoundedTrieData object represents the data stored in a bounded trie. It can either be a
+ * singleton list of strings, or a trie with a given root and bound.
+ *
+ * @param singleton the singleton list of strings, or null if the data is a trie
+ * @param root the root of the trie, or null if the data is a singleton list
+ * @param bound the maximum number of elements allowed in the trie
+ * @throws IllegalArgumentException if both {@code singleton} and {@code root} are non-null
+ */
+ public BoundedTrieData(
+ @Nullable List singleton, @Nullable BoundedTrieNode root, int bound) {
+ assert singleton == null || root == null;
+ this.singleton = singleton;
+ this.root = root;
+ this.bound = bound;
+ }
+
+ /** Converts this {@link BoundedTrieData} to its proto {@link BoundedTrie}. */
+ public synchronized BoundedTrie toProto() {
+ BoundedTrie.Builder builder = BoundedTrie.newBuilder();
+ builder.setBound(this.bound);
+ if (this.singleton != null) {
+ builder.addAllSingleton(this.singleton);
+ }
+ if (this.root != null) {
+ builder.setRoot(this.root.toProto());
+ }
+ return builder.build();
+ }
+
+ /** Creates a {@link BoundedTrieData} instance from its proto {@link BoundedTrie}. */
+ public static BoundedTrieData fromProto(BoundedTrie proto) {
+ if (proto.hasRoot()) {
+ return new BoundedTrieData(
+ null, BoundedTrieNode.fromProto(proto.getRoot()), proto.getBound());
+ } else {
+ return new BoundedTrieData(proto.getSingletonList(), null, proto.getBound());
+ }
+ }
+
+ /** Returns this {@link BoundedTrieData} as a {@link BoundedTrieNode}. */
+ @Nonnull
+ private synchronized BoundedTrieNode asTrie() {
+ if (this.root != null) {
+ return this.root;
+ } else {
+ BoundedTrieNode trieNode = new BoundedTrieNode();
+ if (this.singleton != null) {
+ trieNode.add(this.singleton);
+ }
+ return trieNode;
+ }
+ }
+
+ /** Returns a new {@link BoundedTrieData} instance that is a deep copy of this instance. */
+ public synchronized BoundedTrieData getCumulative() {
+ List singleton = this.singleton == null ? null : new ArrayList<>(this.singleton);
+ // deep copy
+ BoundedTrieNode root = this.root == null ? null : this.root.deepCopy();
+ return new BoundedTrieData(singleton, root, this.bound);
+ }
+
+ /**
+ * Returns {@link BoundedTrieResult}, which represents all path in the bounded trie. The last
+ * element in each path is a boolean in string representation denoting whether this path was
+ * truncated. i.e. <["a", "b", "false"], ["c", "true"]>
+ *
+ * @return The set of paths.
+ */
+ public synchronized BoundedTrieResult extractResult() {
+ if (this.root == null) {
+ if (this.singleton == null) {
+ return BoundedTrieResult.empty();
+ } else {
+ List list = new ArrayList<>(this.singleton);
+ list.add(String.valueOf(false));
+ return BoundedTrieResult.create(Collections.singleton(list));
+ }
+ } else {
+ return BoundedTrieResult.create(new HashSet<>(this.root.flattened()));
+ }
+ }
+
+ /**
+ * Adds a new path to this {@link BoundedTrieData} and hence the {@link BoundedTrieData} is
+ * modified.
+ *
+ * @param segments The path to add.
+ */
+ public synchronized void add(Iterable segments) {
+ List segmentsParts = ImmutableList.copyOf(segments);
+ if (this.root == null) {
+ if (this.singleton == null || !this.singleton.equals(segmentsParts)) {
+ this.root = this.asTrie();
+ this.singleton = null;
+ }
+ }
+
+ if (this.root != null) {
+ this.root.add(segmentsParts);
+ if (this.root.getSize() > this.bound) {
+ this.root.trim();
+ }
+ }
+ }
+
+ /**
+ * Combines this {@link BoundedTrieData} with another {@link BoundedTrieData} by doing a deep
+ * copy.
+ *
+ * @param other The other {@link BoundedTrieData} to combine with.
+ * @return The combined {@link BoundedTrieData}.
+ */
+ public BoundedTrieData combine(@Nonnull BoundedTrieData other) {
+ BoundedTrieData otherDeepCopy;
+ // other can be modified in some different thread, and we need to atomically access
+ // its fields to combine correctly. Furthermore, doing this whole method under
+ // synchronized(other) is not safe as it might lead to deadlock. Assume the current
+ // thread got lock on 'this' and is executing combine with `other` and waiting to get a
+ // lock on `other` while some other thread is performing `other.combiner(this)` and
+ // waiting to get a lock on `this` object.
+ // Here it is safe to get a lock on other as we don't yet hold a lock on this to end up with
+ // race condition.
+ synchronized (other) {
+ if (other.root == null && other.singleton == null) {
+ return this;
+ }
+ otherDeepCopy = other.getCumulative();
+ }
+
+ synchronized (this) {
+ if (this.root == null && this.singleton == null) {
+ return otherDeepCopy;
+ }
+ otherDeepCopy.root = otherDeepCopy.asTrie();
+ otherDeepCopy.singleton = null;
+ otherDeepCopy.root.merge(this.asTrie());
+ otherDeepCopy.bound = Math.min(this.bound, otherDeepCopy.bound);
+ while (otherDeepCopy.root.getSize() > otherDeepCopy.bound) {
+ otherDeepCopy.root.trim();
+ }
+ return otherDeepCopy;
+ }
+ }
+
+ /**
+ * Returns the number of paths stored in this trie.
+ *
+ * @return The size of the trie.
+ */
+ public synchronized int size() {
+ if (this.singleton != null) {
+ return 1;
+ } else if (this.root != null) {
+ return root.getSize();
+ } else {
+ return 0;
+ }
+ }
+
+ public synchronized void clear() {
+ this.root = null;
+ this.singleton = null;
+ this.bound = DEFAULT_BOUND;
+ }
+
+ /**
+ * Checks if the trie contains the given path.
+ *
+ * @param value The path to check.
+ * @return True if the trie contains the path, false otherwise.
+ */
+ public synchronized boolean contains(@Nonnull List value) {
+ if (this.singleton != null) {
+ return value.equals(this.singleton);
+ } else if (this.root != null) {
+ return this.root.contains(value);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public final boolean equals(@Nullable Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (other == null || this.getClass() != other.getClass()) {
+ return false;
+ }
+ BoundedTrieData that = (BoundedTrieData) other;
+ return this.asTrie().equals(that.asTrie());
+ }
+
+ @Override
+ public final int hashCode() {
+ return this.asTrie().hashCode();
+ }
+
+ @Override
+ public final String toString() {
+ return "BoundedTrieData(" + this.asTrie() + ")";
+ }
+
+ // ------------------------------ BoundedTrieNode Implementation ------------------------------
+ /**
+ * BoundedTrieNode implementation. This class is not thread-safe and relies on the {@link
+ * BoundedTrieData} which uses this class to ensure thread-safety by acquiring a lock on the root
+ * of the tree itself. This avoids acquiring and release N nodes in a path. This class is not
+ * intended to be used directly outside of {@link BoundedTrieData} with multiple threads.
+ */
+ @VisibleForTesting
+ static class BoundedTrieNode implements Serializable {
+
+ public static final String TRUNCATED_TRUE = String.valueOf(true);
+ public static final String TRUNCATED_FALSE = String.valueOf(false);
+ /**
+ * A map from strings to child nodes. Each key represents a segment of a path/FQN, and the
+ * corresponding value represents the subtree rooted at that segment.
+ */
+ private Map children;
+
+ /**
+ * A flag indicating whether this node has been truncated. A truncated node represents an
+ * aggregation/roll-up of multiple paths that share a common prefix.
+ */
+ private boolean truncated;
+
+ /**
+ * The size of the subtree rooted at this node. This represents the number of distinct paths
+ * that pass through this node.
+ */
+ private int size;
+
+ /** Constructs an empty `BoundedTrieNode` with size 1 and not truncated. */
+ BoundedTrieNode() {
+ this(new HashMap<>(), false, 1);
+ }
+
+ /**
+ * Constructs a `BoundedTrieNode` with the given children, truncation status, and size.
+ *
+ * @param children The children of this node.
+ * @param truncated Whether this node is truncated.
+ * @param size The size of the subtree rooted at this node.
+ */
+ BoundedTrieNode(Map children, boolean truncated, int size) {
+ this.children = children;
+ this.size = size;
+ this.truncated = truncated;
+ }
+
+ /**
+ * Constructs a deep copy of this `BoundedTrieNode`.
+ *
+ * @return A deep copy of this node.
+ */
+ BoundedTrieNode deepCopy() {
+ BoundedTrieNode copyNode = new BoundedTrieNode();
+ copyNode.truncated = this.truncated;
+ copyNode.size = this.size;
+ copyNode.children = new HashMap<>();
+ // deep copy
+ this.children.forEach((key, value) -> copyNode.children.put(key, value.deepCopy()));
+ return copyNode;
+ }
+
+ /**
+ * Adds a path represented by the given list of segments to this trie.
+ *
+ * @param segments The segments of the path to add.
+ * @return The change in the size of the subtree rooted at this node.
+ */
+ int add(List segments) {
+ if (truncated || segments.isEmpty()) {
+ return 0;
+ }
+ String head = segments.get(0);
+ List tail = segments.subList(1, segments.size());
+ boolean wasEmpty = children.isEmpty();
+ BoundedTrieNode currChild = children.get(head);
+ int delta = 0;
+ if (currChild == null) {
+ currChild = new BoundedTrieNode();
+ children.put(head, currChild);
+ delta = wasEmpty ? 0 : 1;
+ }
+ if (!tail.isEmpty()) {
+ delta += currChild.add(tail);
+ }
+ size += delta;
+ return delta;
+ }
+
+ /**
+ * Adds multiple paths to this trie.
+ *
+ * @param segmentsIter An iterator over the paths to add.
+ * @return The total change in the size of the subtree rooted at this node.
+ */
+ @VisibleForTesting
+ int addAll(List> segmentsIter) {
+ return segmentsIter.stream().mapToInt(this::add).sum();
+ }
+
+ /**
+ * Trims this trie by truncating the largest subtree.
+ *
+ * @return The change in the size of the subtree rooted at this node.
+ */
+ int trim() {
+ if (children.isEmpty()) {
+ return 0;
+ }
+ BoundedTrieNode maxChild =
+ Collections.max(children.values(), Comparator.comparingInt(BoundedTrieNode::getSize));
+ int delta;
+ if (maxChild.size == 1) {
+ delta = 1 - size;
+ truncated = true;
+ children = new HashMap<>();
+ } else {
+ delta = maxChild.trim();
+ }
+ size += delta;
+ return delta;
+ }
+
+ /**
+ * Merges the given `BoundedTrieNode` into this node and as a result this node is changed.
+ *
+ * @param other The node to merge.
+ * @return The change in the size of the subtree rooted at this node.
+ */
+ int merge(BoundedTrieNode other) {
+ if (truncated) {
+ return 0;
+ }
+ if (other.truncated) {
+ truncated = true;
+ children = new HashMap<>();
+ int delta = 1 - size;
+ size += delta;
+ return delta;
+ }
+ if (other.children.isEmpty()) {
+ return 0;
+ }
+ if (children.isEmpty()) {
+ children.putAll(other.children);
+ int delta = other.size - this.size;
+ size += delta;
+ return delta;
+ }
+ int delta = 0;
+ for (Map.Entry entry : other.children.entrySet()) {
+ String prefix = entry.getKey();
+ BoundedTrieNode otherChild = entry.getValue();
+ BoundedTrieNode thisChild = children.get(prefix);
+ if (thisChild == null) {
+ children.put(prefix, otherChild);
+ delta += otherChild.size;
+ } else {
+ delta += thisChild.merge(otherChild);
+ }
+ }
+ size += delta;
+ return delta;
+ }
+
+ /**
+ * Returns a flattened representation of this trie.
+ *
+ * The flattened representation is a list of lists of strings, where each inner list
+ * represents a path in the trie and the last element in the list is a boolean in string
+ * representation denoting whether this path was truncated. i.e. <["a", "b", "false"], ["c",
+ * "true"]>
+ *
+ * @return The flattened representation of this trie.
+ */
+ List> flattened() {
+ List> result = new ArrayList<>();
+ if (truncated) {
+ result.add(Collections.singletonList(TRUNCATED_TRUE));
+ } else if (children.isEmpty()) {
+ result.add(Collections.singletonList(TRUNCATED_FALSE));
+ } else {
+ List prefixes = new ArrayList<>(children.keySet());
+ Collections.sort(prefixes);
+ for (String prefix : prefixes) {
+ BoundedTrieNode child = children.get(prefix);
+ if (child != null) {
+ for (List flattened : child.flattened()) {
+ List newList = new ArrayList<>();
+ newList.add(prefix);
+ newList.addAll(flattened);
+ result.add(newList);
+ }
+ }
+ }
+ }
+ return result;
+ }
+
+ /**
+ * Converts this `BoundedTrieNode` to proto.
+ *
+ * @return The {@link org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode}
+ * representation of this node.
+ */
+ org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode toProto() {
+ org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode.Builder builder =
+ org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode.newBuilder();
+ builder.setTruncated(truncated);
+ children.forEach((key, value) -> builder.putChildren(key, value.toProto()));
+ return builder.build();
+ }
+
+ /**
+ * Constructs a `BoundedTrieNode` from proto.
+ *
+ * @param proto The {@link org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode}
+ * representation of the node.
+ * @return The corresponding `BoundedTrieNode`.
+ */
+ static BoundedTrieNode fromProto(
+ org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode proto) {
+ BoundedTrieNode node = new BoundedTrieNode();
+ if (proto.getTruncated()) {
+ node.truncated = true;
+ node.children = new HashMap<>();
+ } else {
+ node.children = new HashMap<>();
+ proto
+ .getChildrenMap()
+ .forEach((key, value) -> node.children.put(key, BoundedTrieNode.fromProto(value)));
+ node.size =
+ Math.max(1, node.children.values().stream().mapToInt(BoundedTrieNode::getSize).sum());
+ }
+ return node;
+ }
+
+ /**
+ * Checks if the trie contains the given path represented by the list of segments.
+ *
+ * @param segments The segments of the path to check.
+ * @return True if the trie contains the path, false otherwise.
+ */
+ boolean contains(List segments) {
+ if (truncated || segments.isEmpty()) {
+ return true;
+ }
+ String head = segments.get(0);
+ List tail = segments.subList(1, segments.size());
+ return children.containsKey(head) && children.get(head).contains(tail);
+ }
+
+ /**
+ * Returns the size of the subtree rooted at this node.
+ *
+ * @return The size of the subtree.
+ */
+ int getSize() {
+ return size;
+ }
+
+ /**
+ * Returns whether this node is truncated.
+ *
+ * @return Whether this node is truncated.
+ */
+ boolean isTruncated() {
+ return truncated;
+ }
+
+ @Override
+ public int hashCode() {
+ int result = 17; // standard prime numbers
+ result = 31 * result + size;
+ // recursively traverse to calculate hashcode of each node
+ result = 31 * result + children.hashCode();
+ result = 31 * result + (truncated ? 1 : 0);
+ return result;
+ }
+
+ @Override
+ public boolean equals(@Nullable Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (other == null || getClass() != other.getClass()) {
+ return false;
+ }
+ BoundedTrieNode that = (BoundedTrieNode) other;
+ return truncated == that.truncated && children.equals(that.children);
+ }
+
+ @Override
+ public String toString() {
+ return "{"
+ + flattened().stream()
+ .map(list -> "'" + String.join("", list) + "'")
+ .collect(Collectors.joining(", "))
+ + "}";
+ }
+ }
+}
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/DefaultMetricResults.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/DefaultMetricResults.java
index ea8a333d397b..9a9235843836 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/DefaultMetricResults.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/DefaultMetricResults.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.core.metrics;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.GaugeResult;
import org.apache.beam.sdk.metrics.MetricFiltering;
@@ -42,16 +43,19 @@ public class DefaultMetricResults extends MetricResults {
private final Iterable> distributions;
private final Iterable> gauges;
private final Iterable> stringSets;
+ private final Iterable> boundedTries;
public DefaultMetricResults(
Iterable> counters,
Iterable> distributions,
Iterable> gauges,
- Iterable> stringSets) {
+ Iterable> stringSets,
+ Iterable> boundedTries) {
this.counters = counters;
this.distributions = distributions;
this.gauges = gauges;
this.stringSets = stringSets;
+ this.boundedTries = boundedTries;
}
@Override
@@ -62,6 +66,8 @@ public MetricQueryResults queryMetrics(@Nullable MetricsFilter filter) {
distributions, distribution -> MetricFiltering.matches(filter, distribution.getKey())),
Iterables.filter(gauges, gauge -> MetricFiltering.matches(filter, gauge.getKey())),
Iterables.filter(
- stringSets, stringSets -> MetricFiltering.matches(filter, stringSets.getKey())));
+ stringSets, stringSets -> MetricFiltering.matches(filter, stringSets.getKey())),
+ Iterables.filter(
+ boundedTries, boundedTries -> MetricFiltering.matches(filter, boundedTries.getKey())));
}
}
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricUpdates.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricUpdates.java
index ada5bda4df4a..29bf4bf700c5 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricUpdates.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricUpdates.java
@@ -34,6 +34,7 @@ public abstract class MetricUpdates {
Collections.emptyList(),
Collections.emptyList(),
Collections.emptyList(),
+ Collections.emptyList(),
Collections.emptyList());
/**
@@ -66,14 +67,17 @@ public static MetricUpdate create(MetricKey key, T update) {
/** All the sets updates. */
public abstract Iterable> stringSetUpdates();
+ public abstract Iterable> boundedTrieUpdates();
+
/** Create a new {@link MetricUpdates} bundle. */
public static MetricUpdates create(
Iterable> counterUpdates,
Iterable> distributionUpdates,
Iterable> gaugeUpdates,
- Iterable> stringSetUpdates) {
+ Iterable> stringSetUpdates,
+ Iterable> boundedTrieUpdates) {
return new AutoValue_MetricUpdates(
- counterUpdates, distributionUpdates, gaugeUpdates, stringSetUpdates);
+ counterUpdates, distributionUpdates, gaugeUpdates, stringSetUpdates, boundedTrieUpdates);
}
/** Returns true if there are no updates in this MetricUpdates object. */
@@ -81,6 +85,7 @@ public boolean isEmpty() {
return Iterables.isEmpty(counterUpdates())
&& Iterables.isEmpty(distributionUpdates())
&& Iterables.isEmpty(gaugeUpdates())
- && Iterables.isEmpty(stringSetUpdates());
+ && Iterables.isEmpty(stringSetUpdates())
+ && Iterables.isEmpty(boundedTrieUpdates());
}
}
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java
index 99cf98508505..9dcb7a21812f 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java
@@ -17,14 +17,17 @@
*/
package org.apache.beam.runners.core.metrics;
+import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.BOUNDED_TRIE_TYPE;
import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.DISTRIBUTION_INT64_TYPE;
import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.LATEST_INT64_TYPE;
import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.SET_STRING_TYPE;
import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.SUM_INT64_TYPE;
+import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeBoundedTrie;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Counter;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Distribution;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Gauge;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeStringSet;
+import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeBoundedTrie;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeInt64Counter;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeInt64Distribution;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeInt64Gauge;
@@ -93,6 +96,9 @@ public class MetricsContainerImpl implements Serializable, MetricsContainer {
private MetricsMap, HistogramCell> histograms =
new MetricsMap<>(HistogramCell::new);
+ private MetricsMap boundedTries =
+ new MetricsMap<>(BoundedTrieCell::new);
+
private MetricsContainerImpl(@Nullable String stepName, boolean isProcessWide) {
this.stepName = stepName;
this.isProcessWide = isProcessWide;
@@ -129,6 +135,7 @@ public void reset() {
gauges.forEachValue(GaugeCell::reset);
histograms.forEachValue(HistogramCell::reset);
stringSets.forEachValue(StringSetCell::reset);
+ boundedTries.forEachValue(BoundedTrieCell::reset);
}
/**
@@ -216,6 +223,23 @@ public StringSetCell getStringSet(MetricName metricName) {
return stringSets.tryGet(metricName);
}
+ /**
+ * Return a {@code BoundedTrieCell} named {@code metricName}. If it doesn't exist, create a {@code
+ * Metric} with the specified name.
+ */
+ @Override
+ public BoundedTrieCell getBoundedTrie(MetricName metricName) {
+ return boundedTries.get(metricName);
+ }
+
+ /**
+ * Return a {@code BoundedTrieCell} named {@code metricName}. If it doesn't exist, return {@code
+ * null}.
+ */
+ public @Nullable BoundedTrieCell tryGetBoundedTrie(MetricName metricName) {
+ return boundedTries.tryGet(metricName);
+ }
+
private >
ImmutableList> extractUpdates(MetricsMap cells) {
ImmutableList.Builder> updates = ImmutableList.builder();
@@ -238,7 +262,8 @@ public MetricUpdates getUpdates() {
extractUpdates(counters),
extractUpdates(distributions),
extractUpdates(gauges),
- extractUpdates(stringSets));
+ extractUpdates(stringSets),
+ extractUpdates(boundedTries));
}
/** @return The MonitoringInfo metadata from the metric. */
@@ -342,6 +367,15 @@ public MetricUpdates getUpdates() {
MonitoringInfoConstants.Urns.USER_SET_STRING);
}
+ /** @return The MonitoringInfo metadata from the string set metric. */
+ private @Nullable SimpleMonitoringInfoBuilder boundedTrieToMonitoringMetadata(
+ MetricKey metricKey) {
+ return metricToMonitoringMetadata(
+ metricKey,
+ MonitoringInfoConstants.TypeUrns.BOUNDED_TRIE_TYPE,
+ MonitoringInfoConstants.Urns.USER_BOUNDED_TRIE);
+ }
+
/**
* @param metricUpdate
* @return The MonitoringInfo generated from the string set metricUpdate.
@@ -356,6 +390,20 @@ public MetricUpdates getUpdates() {
return builder.build();
}
+ /**
+ * @param metricUpdate
+ * @return The MonitoringInfo generated from the string set metricUpdate.
+ */
+ private @Nullable MonitoringInfo boundedTrieUpdateToMonitoringInfo(
+ MetricUpdate metricUpdate) {
+ SimpleMonitoringInfoBuilder builder = boundedTrieToMonitoringMetadata(metricUpdate.getKey());
+ if (builder == null) {
+ return null;
+ }
+ builder.setBoundedTrieValue(metricUpdate.getUpdate());
+ return builder.build();
+ }
+
/** Return the cumulative values for any metrics in this container as MonitoringInfos. */
@Override
public Iterable getMonitoringInfos() {
@@ -390,6 +438,12 @@ public Iterable getMonitoringInfos() {
monitoringInfos.add(mi);
}
}
+ for (MetricUpdate metricUpdate : metricUpdates.boundedTrieUpdates()) {
+ MonitoringInfo mi = boundedTrieUpdateToMonitoringInfo(metricUpdate);
+ if (mi != null) {
+ monitoringInfos.add(mi);
+ }
+ }
return monitoringInfos;
}
@@ -432,6 +486,16 @@ public Map getMonitoringData(ShortIdMap shortIds) {
}
}
});
+ boundedTries.forEach(
+ (metricName, boundedTrieCell) -> {
+ if (boundedTrieCell.getDirty().beforeCommit()) {
+ String shortId =
+ getShortId(metricName, this::boundedTrieToMonitoringMetadata, shortIds);
+ if (shortId != null) {
+ builder.put(shortId, encodeBoundedTrie(boundedTrieCell.getCumulative()));
+ }
+ }
+ });
return builder.build();
}
@@ -467,6 +531,7 @@ public void commitUpdates() {
distributions.forEachValue(distribution -> distribution.getDirty().afterCommit());
gauges.forEachValue(gauge -> gauge.getDirty().afterCommit());
stringSets.forEachValue(sSets -> sSets.getDirty().afterCommit());
+ boundedTries.forEachValue(bTrie -> bTrie.getDirty().afterCommit());
}
private >
@@ -489,7 +554,8 @@ public MetricUpdates getCumulative() {
extractCumulatives(counters),
extractCumulatives(distributions),
extractCumulatives(gauges),
- extractCumulatives(stringSets));
+ extractCumulatives(stringSets),
+ extractCumulatives(boundedTries));
}
/** Update values of this {@link MetricsContainerImpl} by merging the value of another cell. */
@@ -499,6 +565,7 @@ public void update(MetricsContainerImpl other) {
updateGauges(gauges, other.gauges);
updateHistograms(histograms, other.histograms);
updateStringSets(stringSets, other.stringSets);
+ updateBoundedTrie(boundedTries, other.boundedTries);
}
private void updateForSumInt64Type(MonitoringInfo monitoringInfo) {
@@ -527,6 +594,12 @@ private void updateForStringSetType(MonitoringInfo monitoringInfo) {
stringSet.update(decodeStringSet(monitoringInfo.getPayload()));
}
+ private void updateForBoundedTrieType(MonitoringInfo monitoringInfo) {
+ MetricName metricName = MonitoringInfoMetricName.of(monitoringInfo);
+ BoundedTrieCell boundedTrie = getBoundedTrie(metricName);
+ boundedTrie.update(decodeBoundedTrie(monitoringInfo.getPayload()));
+ }
+
/** Update values of this {@link MetricsContainerImpl} by reading from {@code monitoringInfos}. */
public void update(Iterable monitoringInfos) {
for (MonitoringInfo monitoringInfo : monitoringInfos) {
@@ -551,6 +624,10 @@ public void update(Iterable monitoringInfos) {
updateForStringSetType(monitoringInfo);
break;
+ case BOUNDED_TRIE_TYPE:
+ updateForBoundedTrieType(monitoringInfo);
+ break;
+
default:
LOG.warn("Unsupported metric type {}", monitoringInfo.getType());
}
@@ -585,6 +662,12 @@ private void updateStringSets(
updates.forEach((key, value) -> current.get(key).update(value.getCumulative()));
}
+ private void updateBoundedTrie(
+ MetricsMap current,
+ MetricsMap updates) {
+ updates.forEach((key, value) -> current.get(key).update(value.getCumulative()));
+ }
+
@Override
public boolean equals(@Nullable Object object) {
if (object instanceof MetricsContainerImpl) {
@@ -593,14 +676,15 @@ public boolean equals(@Nullable Object object) {
&& Objects.equals(counters, metricsContainerImpl.counters)
&& Objects.equals(distributions, metricsContainerImpl.distributions)
&& Objects.equals(gauges, metricsContainerImpl.gauges)
- && Objects.equals(stringSets, metricsContainerImpl.stringSets);
+ && Objects.equals(stringSets, metricsContainerImpl.stringSets)
+ && Objects.equals(boundedTries, metricsContainerImpl.boundedTries);
}
return false;
}
@Override
public int hashCode() {
- return Objects.hash(stepName, counters, distributions, gauges, stringSets);
+ return Objects.hash(stepName, counters, distributions, gauges, stringSets, boundedTries);
}
/**
@@ -682,6 +766,16 @@ public String getCumulativeString(@Nullable Set allowedMetricUrns) {
message.append(data.stringSet().toString());
message.append("\n");
}
+ for (Map.Entry cell : boundedTries.entries()) {
+ if (!matchMetric(cell.getKey(), allowedMetricUrns)) {
+ continue;
+ }
+ message.append(cell.getKey().toString());
+ message.append(" = ");
+ BoundedTrieData data = cell.getValue().getCumulative();
+ message.append(data.toString());
+ message.append("\n");
+ }
return message.toString();
}
@@ -690,7 +784,7 @@ public String getCumulativeString(@Nullable Set allowedMetricUrns) {
* this function is to print the changes made to the metrics within a window of time. The
* difference between the counter and histogram bucket counters are calculated between curr and
* prev. The most recent value are used for gauges. Distribution metrics are dropped (As there is
- * meaningful way to calculate the delta). Returns curr if prev is null.
+ * a meaningful way to calculate the delta). Returns curr if prev is null.
*/
public static MetricsContainerImpl deltaContainer(
@Nullable MetricsContainerImpl prev, MetricsContainerImpl curr) {
@@ -726,6 +820,10 @@ public static MetricsContainerImpl deltaContainer(
// Simply take the most recent value for stringSets, no need to count deltas.
deltaContainer.stringSets.get(cell.getKey()).update(cell.getValue().getCumulative());
}
+ for (Map.Entry cell : curr.boundedTries.entries()) {
+ // Simply take the most recent value for boundedTries, no need to count deltas.
+ deltaContainer.boundedTries.get(cell.getKey()).update(cell.getValue().getCumulative());
+ }
return deltaContainer;
}
}
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMap.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMap.java
index 688491184e67..748e905396bf 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMap.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMap.java
@@ -137,6 +137,7 @@ public static MetricResults asMetricResults(
Map> distributions = new HashMap<>();
Map> gauges = new HashMap<>();
Map> sets = new HashMap<>();
+ Map> boundedTries = new HashMap<>();
attemptedMetricsContainers.forEachMetricContainer(
container -> {
@@ -146,6 +147,8 @@ public static MetricResults asMetricResults(
distributions, cumulative.distributionUpdates(), DistributionData::combine);
mergeAttemptedResults(gauges, cumulative.gaugeUpdates(), GaugeData::combine);
mergeAttemptedResults(sets, cumulative.stringSetUpdates(), StringSetData::combine);
+ mergeAttemptedResults(
+ boundedTries, cumulative.boundedTrieUpdates(), BoundedTrieData::combine);
});
committedMetricsContainers.forEachMetricContainer(
container -> {
@@ -155,6 +158,8 @@ public static MetricResults asMetricResults(
distributions, cumulative.distributionUpdates(), DistributionData::combine);
mergeCommittedResults(gauges, cumulative.gaugeUpdates(), GaugeData::combine);
mergeCommittedResults(sets, cumulative.stringSetUpdates(), StringSetData::combine);
+ mergeCommittedResults(
+ boundedTries, cumulative.boundedTrieUpdates(), BoundedTrieData::combine);
});
return new DefaultMetricResults(
@@ -167,6 +172,9 @@ public static MetricResults asMetricResults(
.collect(toList()),
sets.values().stream()
.map(result -> result.transform(StringSetData::extractResult))
+ .collect(toList()),
+ boundedTries.values().stream()
+ .map(result -> result.transform(BoundedTrieData::extractResult))
.collect(toList()));
}
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java
index 2bb935111d38..e6758700b755 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java
@@ -54,6 +54,8 @@ public static final class Urns {
extractUrn(MonitoringInfoSpecs.Enum.USER_DISTRIBUTION_DOUBLE);
public static final String USER_SET_STRING =
extractUrn(MonitoringInfoSpecs.Enum.USER_SET_STRING);
+ public static final String USER_BOUNDED_TRIE =
+ extractUrn(MonitoringInfoSpecs.Enum.USER_BOUNDED_TRIE);
public static final String SAMPLED_BYTE_SIZE =
extractUrn(MonitoringInfoSpecs.Enum.SAMPLED_BYTE_SIZE);
public static final String WORK_COMPLETED = extractUrn(MonitoringInfoSpecs.Enum.WORK_COMPLETED);
@@ -165,6 +167,7 @@ public static final class TypeUrns {
public static final String BOTTOM_N_DOUBLE_TYPE = "beam:metrics:bottom_n_double:v1";
public static final String PROGRESS_TYPE = "beam:metrics:progress:v1";
public static final String SET_STRING_TYPE = "beam:metrics:set_string:v1";
+ public static final String BOUNDED_TRIE_TYPE = "beam:metrics:bounded_trie:v1";
static {
// Validate that compile time constants match the values stored in the protos.
@@ -191,6 +194,8 @@ public static final class TypeUrns {
BOTTOM_N_DOUBLE_TYPE.equals(getUrn(MonitoringInfoTypeUrns.Enum.BOTTOM_N_DOUBLE_TYPE)));
checkArgument(PROGRESS_TYPE.equals(getUrn(MonitoringInfoTypeUrns.Enum.PROGRESS_TYPE)));
checkArgument(SET_STRING_TYPE.equals(getUrn(MonitoringInfoTypeUrns.Enum.SET_STRING_TYPE)));
+ checkArgument(
+ BOUNDED_TRIE_TYPE.equals(getUrn(MonitoringInfoTypeUrns.Enum.BOUNDED_TRIE_TYPE)));
}
}
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java
index 88136a864d03..0cd2c48bcb87 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java
@@ -31,6 +31,7 @@
import java.util.Iterator;
import java.util.List;
import java.util.Set;
+import org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrie;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.DoubleCoder;
import org.apache.beam.sdk.coders.IterableCoder;
@@ -39,6 +40,7 @@
import org.apache.beam.sdk.util.ByteStringOutputStream;
import org.apache.beam.sdk.util.HistogramData;
import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets;
import org.joda.time.Instant;
@@ -139,6 +141,20 @@ public static StringSetData decodeStringSet(ByteString payload) {
}
}
+ /** Encodes to {@link MonitoringInfoConstants.TypeUrns#BOUNDED_TRIE_TYPE}. */
+ public static ByteString encodeBoundedTrie(BoundedTrieData data) {
+ return data.toProto().toByteString();
+ }
+
+ /** Decodes from {@link MonitoringInfoConstants.TypeUrns#BOUNDED_TRIE_TYPE}. */
+ public static BoundedTrieData decodeBoundedTrie(ByteString payload) {
+ try {
+ return BoundedTrieData.fromProto(BoundedTrie.parseFrom(payload));
+ } catch (InvalidProtocolBufferException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
/** Encodes to {@link MonitoringInfoConstants.TypeUrns#SUM_INT64_TYPE}. */
public static ByteString encodeInt64Counter(long value) {
ByteStringOutputStream output = new ByteStringOutputStream();
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleMonitoringInfoBuilder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleMonitoringInfoBuilder.java
index e0f5092e6b1f..3d3d7070df76 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleMonitoringInfoBuilder.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleMonitoringInfoBuilder.java
@@ -18,6 +18,7 @@
package org.apache.beam.runners.core.metrics;
import static org.apache.beam.model.pipeline.v1.MetricsApi.monitoringInfoSpec;
+import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeBoundedTrie;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeDoubleCounter;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeDoubleDistribution;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeInt64Counter;
@@ -159,6 +160,16 @@ public SimpleMonitoringInfoBuilder setStringSetValue(StringSetData value) {
return this;
}
+ /**
+ * Encodes the value and sets the type to {@link
+ * MonitoringInfoConstants.TypeUrns#BOUNDED_TRIE_TYPE}.
+ */
+ public SimpleMonitoringInfoBuilder setBoundedTrieValue(BoundedTrieData value) {
+ this.builder.setPayload(encodeBoundedTrie(value));
+ this.builder.setType(MonitoringInfoConstants.TypeUrns.BOUNDED_TRIE_TYPE);
+ return this;
+ }
+
/** Sets the MonitoringInfo label to the given name and value. */
public SimpleMonitoringInfoBuilder setLabel(String labelName, String labelValue) {
this.builder.putLabels(labelName, labelValue);
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/BoundedTrieCellTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/BoundedTrieCellTest.java
new file mode 100644
index 000000000000..b5dcd03fb42c
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/BoundedTrieCellTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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.runners.core.metrics;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
+import org.apache.beam.sdk.metrics.MetricName;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+import org.junit.Assert;
+import org.junit.Test;
+
+/** Tests for {@link BoundedTrieCell}. */
+public class BoundedTrieCellTest {
+ private final BoundedTrieCell cell = new BoundedTrieCell(MetricName.named("lineage", "sources"));
+
+ @Test
+ public void testDeltaAndCumulative() {
+ cell.add("a");
+ cell.add("b", "c");
+ cell.add("b", "d");
+ cell.add("a", "a");
+ BoundedTrieData cumulative = cell.getCumulative();
+ assertEquals(
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ Arrays.asList("a", "a", String.valueOf(false)),
+ Arrays.asList("b", "d", String.valueOf(false)),
+ Arrays.asList("b", "c", String.valueOf(false)))),
+ cumulative.extractResult());
+
+ assertThat(cell.getDirty().beforeCommit(), equalTo(true));
+ cell.getDirty().afterCommit();
+ assertThat(cell.getDirty().beforeCommit(), equalTo(false));
+
+ // adding new value changes the cell
+ cell.add("b", "a");
+ BoundedTrieData newCumulative = cell.getCumulative();
+ assertEquals(
+ newCumulative.extractResult(),
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ Arrays.asList("a", "a", String.valueOf(false)),
+ Arrays.asList("b", "d", String.valueOf(false)),
+ Arrays.asList("b", "c", String.valueOf(false)),
+ Arrays.asList("b", "a", String.valueOf(false)))));
+
+ // but not previously obtained cumulative value
+ assertEquals(
+ cumulative.extractResult(),
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ Arrays.asList("a", "a", String.valueOf(false)),
+ Arrays.asList("b", "d", String.valueOf(false)),
+ Arrays.asList("b", "c", String.valueOf(false)))));
+
+ assertThat(
+ "Adding a new value made the cell dirty", cell.getDirty().beforeCommit(), equalTo(true));
+ }
+
+ @Test
+ public void testEquals() {
+ BoundedTrieCell boundedTrieCell = new BoundedTrieCell(MetricName.named("namespace", "name"));
+ BoundedTrieCell equal = new BoundedTrieCell(MetricName.named("namespace", "name"));
+ assertEquals(boundedTrieCell, equal);
+ assertEquals(boundedTrieCell.hashCode(), equal.hashCode());
+ }
+
+ @Test
+ public void testNotEquals() {
+ BoundedTrieCell boundedTrieCell = new BoundedTrieCell(MetricName.named("namespace", "name"));
+
+ Assert.assertNotEquals(boundedTrieCell, new Object());
+
+ BoundedTrieCell differentDirty = new BoundedTrieCell(MetricName.named("namespace", "name"));
+ differentDirty.getDirty().afterModification();
+ Assert.assertNotEquals(boundedTrieCell, differentDirty);
+ Assert.assertNotEquals(boundedTrieCell.hashCode(), differentDirty.hashCode());
+
+ BoundedTrieCell differentTrieCell = new BoundedTrieCell(MetricName.named("namespace", "name"));
+ BoundedTrieData updateData = new BoundedTrieData(ImmutableList.of("hello"));
+ differentTrieCell.update(updateData);
+ Assert.assertNotEquals(boundedTrieCell, differentTrieCell);
+ Assert.assertNotEquals(boundedTrieCell.hashCode(), differentTrieCell.hashCode());
+
+ BoundedTrieCell differentName = new BoundedTrieCell(MetricName.named("DIFFERENT", "DIFFERENT"));
+ Assert.assertNotEquals(boundedTrieCell, differentName);
+ Assert.assertNotEquals(boundedTrieCell.hashCode(), differentName.hashCode());
+ }
+
+ @Test
+ public void testReset() {
+ BoundedTrieCell boundedTrieCell = new BoundedTrieCell(MetricName.named("namespace", "name"));
+ boundedTrieCell.add("hello");
+ Assert.assertNotEquals(boundedTrieCell.getDirty(), new DirtyState());
+ assertThat(
+ boundedTrieCell.getCumulative(), equalTo(new BoundedTrieData(ImmutableList.of("hello"))));
+
+ boundedTrieCell.reset();
+ assertThat(boundedTrieCell.getCumulative(), equalTo(new BoundedTrieData()));
+ assertThat(boundedTrieCell.getDirty(), equalTo(new DirtyState()));
+ }
+
+ @Test(timeout = 5000)
+ public void testBoundedTrieDiffrentThreadWrite() throws InterruptedException {
+ BoundedTrieCell cell = new BoundedTrieCell(MetricName.named("namespace", "name"));
+ AtomicBoolean finished = new AtomicBoolean(false);
+ Thread increment =
+ new Thread(
+ () -> {
+ for (long i = 0; !finished.get(); ++i) {
+ cell.add(String.valueOf(i));
+ try {
+ Thread.sleep(1);
+ } catch (InterruptedException e) {
+ break;
+ }
+ }
+ });
+ increment.start();
+ Instant start = Instant.now();
+ try {
+ while (true) {
+ BoundedTrieData cumulative = cell.getCumulative();
+ if (Instant.now().isAfter(start.plusSeconds(3)) && cumulative.size() > 0) {
+ finished.compareAndSet(false, true);
+ break;
+ }
+ }
+ } finally {
+ increment.interrupt();
+ increment.join();
+ }
+
+ BoundedTrieData s = cell.getCumulative();
+ for (long i = 0; i < s.size(); ++i) {
+ assertTrue(s.contains(ImmutableList.of(String.valueOf(i))));
+ }
+ }
+}
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/BoundedTrieNodeTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/BoundedTrieNodeTest.java
new file mode 100644
index 000000000000..ca5421e6f77f
--- /dev/null
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/BoundedTrieNodeTest.java
@@ -0,0 +1,1017 @@
+/*
+ * 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.runners.core.metrics;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+import org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrie;
+import org.apache.beam.runners.core.metrics.BoundedTrieData.BoundedTrieNode;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Tests for {@link BoundedTrieNode}. */
+@RunWith(JUnit4.class)
+public class BoundedTrieNodeTest {
+ private static final Logger LOG = LoggerFactory.getLogger(BoundedTrieNodeTest.class);
+
+ /**
+ * Generates {@code n} random segments with a fixed depth.
+ *
+ * @param n The number of segments to generate.
+ * @param depth The depth of each segment.
+ * @param overlap The probability that a segment will share a prefix with a previous segment.
+ * @param rand A random number generator.
+ * @return A list of segments.
+ */
+ private static List> generateSegmentsFixedDepth(
+ int n, int depth, double overlap, Random rand) {
+ if (depth == 0) {
+ return Collections.nCopies(n, Collections.emptyList());
+ } else {
+ List> result = new ArrayList<>();
+ List seen = new ArrayList<>();
+ for (List suffix : generateSegmentsFixedDepth(n, depth - 1, overlap, rand)) {
+ String prefix;
+ if (seen.isEmpty() || rand.nextDouble() > overlap) {
+ prefix = String.valueOf((char) ('a' + seen.size()));
+ seen.add(prefix);
+ } else {
+ prefix = seen.get(rand.nextInt(seen.size()));
+ }
+ List newSegments = new ArrayList<>();
+ newSegments.add(prefix);
+ newSegments.addAll(suffix);
+ result.add(newSegments);
+ }
+ return result;
+ }
+ }
+
+ /**
+ * Generates {@code n} random segments with a depth between {@code minDepth} and {@code maxDepth}.
+ *
+ * @param n The number of segments to generate.
+ * @param minDepth The minimum depth of each segment.
+ * @param maxDepth The maximum depth of each segment.
+ * @param overlap The probability that a segment will share a prefix with a previous segment.
+ * @param rand A random number generator.
+ * @return A list of segments.
+ */
+ private static List> randomSegments(
+ int n, int minDepth, int maxDepth, double overlap, Random rand) {
+ List> result = new ArrayList<>();
+ List depths = new ArrayList<>();
+ for (int i = 0; i < n; i++) {
+ depths.add(minDepth + (i % (maxDepth - minDepth + 1)));
+ }
+ Iterator depthIterator = depths.iterator();
+ for (List segments : generateSegmentsFixedDepth(n, maxDepth, overlap, rand)) {
+ int depth = depthIterator.next();
+ result.add(segments.subList(0, depth));
+ }
+ return result;
+ }
+
+ /**
+ * Asserts that the given {@link BoundedTrieNode} covers the expected segments.
+ *
+ * @param node The {@link BoundedTrieNode} to check.
+ * @param expected The expected segments.
+ * @param maxTruncated The maximum number of truncated segments allowed.
+ */
+ private void assertCovers(BoundedTrieNode node, Set> expected, int maxTruncated) {
+ assertCoversFlattened(node.flattened(), expected, maxTruncated);
+ }
+
+ /**
+ * Verifies that the flattened list of segments covers the expected segments.
+ *
+ * @param flattened The flattened list of segments.
+ * @param expected The expected segments.
+ * @param maxTruncated The maximum number of truncated segments allowed.
+ */
+ private void assertCoversFlattened(
+ List> flattened, Set> expected, int maxTruncated) {
+ Set> exactSegments = new HashSet<>();
+ Set> truncatedSegments = new HashSet<>();
+ for (List entry : flattened) {
+ List segments = new ArrayList<>(entry.subList(0, entry.size() - 1));
+ String last = entry.get(entry.size() - 1);
+ if (Boolean.parseBoolean(last)) {
+ truncatedSegments.add(segments);
+ } else {
+ exactSegments.add(segments);
+ }
+ }
+
+ assertTrue(
+ "Exact segments set should not be larger than expected set",
+ exactSegments.size() <= expected.size());
+ assertTrue(
+ "Expected set should contain all exact segments", expected.containsAll(exactSegments));
+
+ assertTrue(
+ "Number of truncated segments should not exceed maxTruncated",
+ truncatedSegments.size() <= maxTruncated);
+
+ Set> seenTruncated = new HashSet<>();
+ for (List segments : expected) {
+ if (!exactSegments.contains(segments)) {
+ int found = 0;
+ for (int i = 0; i < segments.size(); i++) {
+ if (truncatedSegments.contains(segments.subList(0, i))) {
+ seenTruncated.add(segments.subList(0, i));
+ found++;
+ }
+ }
+ assertEquals(
+ String.format(
+ "Expected exactly one prefix of %s to occur in %s, found %s",
+ segments, truncatedSegments, found),
+ 1,
+ found);
+ }
+ }
+
+ assertEquals(
+ "Seen truncated segments should match the truncated segments set",
+ seenTruncated,
+ truncatedSegments);
+ }
+
+ /**
+ * Runs a test case for the {@link #assertCoversFlattened} method.
+ *
+ * @param flattened The flattened list of segments.
+ * @param expected The expected segments.
+ * @param maxTruncated The maximum number of truncated segments allowed.
+ */
+ private void runCoversTest(List flattened, List expected, int maxTruncated) {
+ List> parsedFlattened =
+ flattened.stream()
+ .map(
+ s -> {
+ List result =
+ new ArrayList<>(Arrays.asList(s.replace("*", "").split("")));
+ result.add(s.endsWith("*") ? Boolean.TRUE.toString() : Boolean.FALSE.toString());
+ return result;
+ })
+ .collect(Collectors.toList());
+ Set> parsedExpected =
+ expected.stream().map(s -> Arrays.asList(s.split(""))).collect(Collectors.toSet());
+ assertCoversFlattened(parsedFlattened, parsedExpected, maxTruncated);
+ }
+
+ private Set> everythingDeduped(Set> everything) {
+ Set> allPrefixes = new HashSet<>();
+ for (List segments : everything) {
+ for (int i = 0; i < segments.size(); i++) {
+ allPrefixes.add(segments.subList(0, i));
+ }
+ }
+ Set> everythingDeduped = new HashSet<>(everything);
+ everythingDeduped.removeAll(allPrefixes);
+ return everythingDeduped;
+ }
+
+ /**
+ * Runs a test case for the {@link BoundedTrieNode} class.
+ *
+ * @param toAdd The segments to add to the {@link BoundedTrieNode}.
+ */
+ private void runTest(List> toAdd) {
+ Set> everything = new HashSet<>(toAdd);
+ Set> everythingDeduped = everythingDeduped(everything);
+
+ // Test basic addition.
+ BoundedTrieNode node = new BoundedTrieNode();
+ int initialSize = node.getSize();
+ assertEquals(1, initialSize);
+
+ int totalSize = initialSize;
+ for (List segments : everything) {
+ totalSize += node.add(segments);
+ }
+
+ assertEquals(everythingDeduped.size(), node.getSize());
+ assertEquals(totalSize, node.getSize());
+ assertCovers(node, everythingDeduped, 0);
+
+ // Test merging.
+ BoundedTrieNode node0 = new BoundedTrieNode();
+ BoundedTrieNode node1 = new BoundedTrieNode();
+ int i = 0;
+ for (List segments : everything) {
+ if (i % 2 == 0) {
+ node0.add(segments);
+ } else {
+ node1.add(segments);
+ }
+ i++;
+ }
+ int preMergeSize = node0.getSize();
+ int mergeDelta = node0.merge(node1);
+ assertEquals(preMergeSize + mergeDelta, node0.getSize());
+ assertEquals(node0, node);
+
+ // Test trimming.
+ int trimDelta = 0;
+ if (node.getSize() > 1) {
+ trimDelta = node.trim();
+ assertTrue(trimDelta < 0);
+ assertEquals(totalSize + trimDelta, node.getSize());
+ assertCovers(node, everythingDeduped, 1);
+ }
+
+ if (node.getSize() > 1) {
+ int trim2Delta = node.trim();
+ assertTrue(trim2Delta < 0);
+ assertEquals(totalSize + trimDelta + trim2Delta, node.getSize());
+ assertCovers(node, everythingDeduped, 2);
+ }
+
+ // Verify adding after trimming is a no-op.
+ BoundedTrieNode nodeCopy = node.deepCopy();
+ for (List segments : everything) {
+ assertEquals(0, node.add(segments));
+ }
+ assertEquals(node, nodeCopy);
+
+ // Verify merging after trimming is a no-op.
+ assertEquals(0, node.merge(node0));
+ assertEquals(0, node.merge(node1));
+ assertEquals(node, nodeCopy);
+
+ // Test adding new values.
+ int expectedDelta = node.isTruncated() ? 0 : 2;
+ List> newValues =
+ Arrays.asList(Collections.singletonList("new1"), Arrays.asList("new2", "new2.1"));
+ assertEquals(expectedDelta, node.addAll(newValues));
+
+ Set> expectedWithNewValues = new HashSet<>(everythingDeduped);
+ expectedWithNewValues.addAll(newValues);
+ assertCovers(node, expectedWithNewValues, 2);
+
+ // Test merging new values.
+ BoundedTrieNode newValuesNode = new BoundedTrieNode();
+ newValuesNode.addAll(newValues);
+ assertCovers(newValuesNode, new HashSet<>(newValues), 0);
+ assertEquals(expectedDelta, nodeCopy.merge(newValuesNode));
+ assertCovers(nodeCopy, expectedWithNewValues, 2);
+ // adding after merge should not change previous node on which this was merged
+ List additionalValue = Arrays.asList("new3", "new3.1");
+ expectedDelta = newValuesNode.isTruncated() ? 0 : 1;
+ assertEquals(expectedDelta, newValuesNode.add(additionalValue));
+ // previous node on which the merge was done should have remained same
+ assertCovers(nodeCopy, expectedWithNewValues, 2);
+ // the newValuesNode should have changed
+ Set> updatedNewValues = new HashSet<>(newValues);
+ updatedNewValues.add(additionalValue);
+ assertCovers(newValuesNode, updatedNewValues, 0);
+ }
+
+ /**
+ * Fuzzy segment generator for testing {@link BoundedTrieNode} class.
+ *
+ * @param iterations The number of iterations to run.
+ * @param n The number of segments to generate for each iteration.
+ * @param minDepth The minimum depth of each segment.
+ * @param maxDepth The maximum depth of each segment.
+ * @param overlap The probability that a segment will share a prefix with a previous segment.
+ */
+ private void runFuzz(int iterations, int n, int minDepth, int maxDepth, double overlap) {
+ for (int i = 0; i < iterations; i++) {
+ long seed = new Random().nextLong();
+ Random rand = new Random(seed);
+ List> segments = randomSegments(n, minDepth, maxDepth, overlap, rand);
+ try {
+ runTest(segments);
+ } catch (AssertionError e) {
+ LOG.info("SEED: {}", seed);
+ throw e;
+ }
+ }
+ }
+
+ @Test
+ public void testTrivial() {
+ runTest(Arrays.asList(Arrays.asList("a", "b"), Arrays.asList("a", "c")));
+ }
+
+ @Test
+ public void testFlat() {
+ runTest(
+ Arrays.asList(Arrays.asList("a", "a"), Arrays.asList("b", "b"), Arrays.asList("c", "c")));
+ }
+
+ @Test
+ public void testDeep() {
+ runTest(Arrays.asList(Collections.nCopies(10, "a"), Collections.nCopies(12, "b")));
+ }
+
+ @Test
+ public void testSmall() {
+ runFuzz(10, 5, 2, 3, 0.5);
+ }
+
+ @Test
+ public void testMedium() {
+ runFuzz(10, 20, 2, 4, 0.5);
+ }
+
+ @Test
+ public void testLargeSparse() {
+ runFuzz(10, 120, 2, 4, 0.2);
+ }
+
+ @Test
+ public void testLargeDense() {
+ runFuzz(10, 120, 2, 4, 0.8);
+ }
+
+ @Test
+ public void testCoversExact() {
+ runCoversTest(Arrays.asList("ab", "ac", "cd"), Arrays.asList("ab", "ac", "cd"), 0);
+
+ assertThrows(
+ AssertionError.class,
+ () -> runCoversTest(Arrays.asList("ab", "ac", "cd"), Arrays.asList("ac", "cd"), 0));
+ assertThrows(
+ AssertionError.class,
+ () -> runCoversTest(Arrays.asList("ab", "ac"), Arrays.asList("ab", "ac", "cd"), 0));
+ assertThrows(
+ AssertionError.class,
+ () -> runCoversTest(Arrays.asList("a*", "cd"), Arrays.asList("ab", "ac", "cd"), 0));
+ }
+
+ @Test
+ public void testCoversTruncated() {
+ runCoversTest(Arrays.asList("a*", "cd"), Arrays.asList("ab", "ac", "cd"), 1);
+ runCoversTest(Arrays.asList("a*", "cd"), Arrays.asList("ab", "ac", "abcde", "cd"), 1);
+
+ assertThrows(
+ AssertionError.class,
+ () -> runCoversTest(Arrays.asList("ab", "ac", "cd"), Arrays.asList("ac", "cd"), 1));
+ assertThrows(
+ AssertionError.class,
+ () -> runCoversTest(Arrays.asList("ab", "ac"), Arrays.asList("ab", "ac", "cd"), 1));
+ assertThrows(
+ AssertionError.class,
+ () -> runCoversTest(Arrays.asList("a*", "c*"), Arrays.asList("ab", "ac", "cd"), 1));
+ assertThrows(
+ AssertionError.class,
+ () -> runCoversTest(Arrays.asList("a*", "c*"), Arrays.asList("ab", "ac"), 1));
+ }
+
+ @Test
+ public void testBoundedTrieDataCombine() {
+ BoundedTrieData empty = new BoundedTrieData();
+ BoundedTrieData singletonA = new BoundedTrieData(ImmutableList.of("a", "a"));
+ BoundedTrieData singletonB = new BoundedTrieData(ImmutableList.of("b", "b"));
+ BoundedTrieNode lotsRoot = new BoundedTrieNode();
+ lotsRoot.addAll(Arrays.asList(Arrays.asList("c", "c"), Arrays.asList("d", "d")));
+ BoundedTrieData lots = new BoundedTrieData(lotsRoot);
+
+ assertEquals(BoundedTrieResult.empty(), empty.extractResult());
+ empty = empty.combine(singletonA);
+ assertEquals(
+ BoundedTrieResult.create(ImmutableSet.of(Arrays.asList("a", "a", String.valueOf(false)))),
+ empty.extractResult());
+ singletonA = singletonA.combine(empty);
+ assertEquals(
+ BoundedTrieResult.create(ImmutableSet.of(Arrays.asList("a", "a", String.valueOf(false)))),
+ singletonA.extractResult());
+ singletonA = singletonA.combine(singletonB);
+ assertEquals(
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ Arrays.asList("a", "a", String.valueOf(false)),
+ Arrays.asList("b", "b", String.valueOf(false)))),
+ singletonA.extractResult());
+ singletonA = singletonA.combine(lots);
+ assertEquals(
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ Arrays.asList("a", "a", String.valueOf(false)),
+ Arrays.asList("b", "b", String.valueOf(false)),
+ Arrays.asList("c", "c", String.valueOf(false)),
+ Arrays.asList("d", "d", String.valueOf(false)))),
+ singletonA.extractResult());
+ lots = lots.combine(singletonA);
+ assertEquals(
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ Arrays.asList("a", "a", String.valueOf(false)),
+ Arrays.asList("b", "b", String.valueOf(false)),
+ Arrays.asList("c", "c", String.valueOf(false)),
+ Arrays.asList("d", "d", String.valueOf(false)))),
+ lots.extractResult());
+ }
+
+ @Test
+ public void testBoundedTrieDataCombineTrim() {
+ BoundedTrieNode left = new BoundedTrieNode();
+ left.addAll(Arrays.asList(Arrays.asList("a", "x"), Arrays.asList("b", "d")));
+ BoundedTrieNode right = new BoundedTrieNode();
+ right.addAll(Arrays.asList(Arrays.asList("a", "y"), Arrays.asList("c", "d")));
+
+ BoundedTrieData mainTree = new BoundedTrieData(null, left, 10);
+ mainTree = mainTree.combine(new BoundedTrieData(null, right, 3));
+
+ assertEquals(
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ Arrays.asList("a", String.valueOf(true)),
+ Arrays.asList("b", "d", String.valueOf(false)),
+ Arrays.asList("c", "d", String.valueOf(false)))),
+ mainTree.extractResult());
+ }
+
+ @Test
+ public void testAddMultiThreaded() throws InterruptedException {
+ final int numThreads = 10;
+ final BoundedTrieData mainTrie = new BoundedTrieData();
+ final CountDownLatch latch = new CountDownLatch(numThreads);
+ ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+
+ Random rand = new Random(new Random().nextLong());
+ List> segments = randomSegments(numThreads, 3, 9, 0.5, rand);
+
+ for (int curThread = 0; curThread < numThreads; curThread++) {
+ int finalCurThread = curThread;
+ executor.execute(
+ () -> {
+ try {
+ mainTrie.add(segments.get(finalCurThread));
+ // }
+ } finally {
+ latch.countDown();
+ }
+ });
+ }
+
+ latch.await();
+ executor.shutdown();
+ assertTrue(executor.awaitTermination(30, TimeUnit.SECONDS));
+ HashSet> dedupedSegments = new HashSet<>(segments);
+ assertEquals(everythingDeduped(dedupedSegments).size(), mainTrie.size());
+ // Assert that all added paths are present in the mainTrie
+ for (List seg : dedupedSegments) {
+ assertTrue(mainTrie.contains(seg));
+ }
+ }
+
+ @Test
+ public void testCombineMultiThreaded() throws InterruptedException {
+ final int numThreads = 10;
+ AtomicReference mainTrie = new AtomicReference<>(new BoundedTrieData());
+ final CountDownLatch latch = new CountDownLatch(numThreads);
+ ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+ Random rand = new Random(new Random().nextLong());
+
+ // initialize mainTrie
+ List initialSegment = randomSegments(1, 3, 9, 0.5, rand).get(0);
+ mainTrie.get().add(initialSegment);
+
+ // prepare all segments in advance outside of multiple threads
+ List> segments = randomSegments(numThreads, 3, 9, 0.5, rand);
+ segments.add(initialSegment);
+ List anotherSegment = randomSegments(1, 3, 9, 0.5, rand).get(0);
+ segments.add(anotherSegment);
+
+ for (int curThread = 0; curThread < numThreads; curThread++) {
+ int finalCurThread = curThread;
+ executor.execute(
+ () -> {
+ try {
+ BoundedTrieData other = new BoundedTrieData();
+ // only reads of segments; no write should be done here
+ other.add(segments.get(finalCurThread));
+ // for one node we add more than one segment to trigger root over
+ // singleton and test combine with root.
+ if (finalCurThread == 7) { // just a randomly selected prime number
+ other.add(anotherSegment);
+ }
+ BoundedTrieData original;
+ do {
+ original = mainTrie.get();
+ } while (!mainTrie.compareAndSet(original, original.combine(other)));
+ // }
+ } finally {
+ latch.countDown();
+ }
+ });
+ }
+
+ latch.await();
+ executor.shutdown();
+ assertTrue(executor.awaitTermination(30, TimeUnit.SECONDS));
+ HashSet> dedupedSegments = new HashSet<>(segments);
+ assertEquals(everythingDeduped(dedupedSegments).size(), mainTrie.get().size());
+ // Assert that all added paths are present in the mainTrie
+ for (List seg : dedupedSegments) {
+ assertTrue(mainTrie.get().contains(seg));
+ }
+ }
+
+ @Test
+ public void testTrim() {
+ BoundedTrieNode root = new BoundedTrieNode();
+ root.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "b", "d"),
+ Arrays.asList("a", "e"))));
+ assertEquals(3, root.getSize());
+ assertEquals(-1, root.trim());
+ assertEquals(2, root.getSize());
+ List> flattened = root.flattened();
+ assertEquals(2, flattened.size());
+ assertFalse(root.isTruncated());
+ assertEquals(
+ flattened,
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", BoundedTrieNode.TRUNCATED_TRUE),
+ Arrays.asList("a", "e", BoundedTrieNode.TRUNCATED_FALSE))));
+ }
+
+ @Test
+ public void testMerge() {
+ BoundedTrieNode root1 = new BoundedTrieNode();
+ root1.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "b", "d"),
+ Arrays.asList("a", "e"))));
+
+ BoundedTrieNode root2 = new BoundedTrieNode();
+ root2.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "f"),
+ Arrays.asList("a", "g"),
+ Collections.singletonList("h"))));
+
+ assertEquals(3, root1.merge(root2));
+ assertEquals(6, root1.getSize());
+ }
+
+ @Test
+ public void testMergeWithTruncatedNode() {
+ BoundedTrieNode root1 = new BoundedTrieNode();
+ root1.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "b", "d"),
+ Arrays.asList("a", "e"))));
+
+ BoundedTrieNode root2 = new BoundedTrieNode();
+ root2.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "f"),
+ Arrays.asList("a", "g"),
+ Collections.singletonList("h"))));
+ root2.trim();
+ List> expected =
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", BoundedTrieNode.TRUNCATED_TRUE),
+ Arrays.asList("h", BoundedTrieNode.TRUNCATED_FALSE)));
+ assertEquals(expected, root2.flattened());
+
+ assertEquals(-1, root1.merge(root2));
+ assertEquals(2, root1.getSize());
+ assertEquals(expected, root1.flattened());
+ }
+
+ @Test
+ public void testMergeWithEmptyNode() {
+ BoundedTrieNode root1 = new BoundedTrieNode();
+ root1.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "b", "d"),
+ Arrays.asList("a", "e"))));
+
+ BoundedTrieNode root2 = new BoundedTrieNode();
+
+ assertEquals(0, root1.merge(root2));
+ assertEquals(3, root1.getSize());
+ assertFalse(root1.isTruncated());
+ }
+
+ @Test
+ public void testMergeOnEmptyNode() {
+ BoundedTrieNode root1 = new BoundedTrieNode();
+ BoundedTrieNode root2 = new BoundedTrieNode();
+ root2.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "b", "d"),
+ Arrays.asList("a", "e"))));
+
+ assertEquals(2, root1.merge(root2));
+ assertEquals(3, root1.getSize());
+ assertFalse(root1.isTruncated());
+ }
+
+ @Test
+ public void testFlattenedWithTruncatedNode() {
+ BoundedTrieNode root = new BoundedTrieNode();
+ root.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "e"),
+ Arrays.asList("a", "b", "d"))));
+ root.trim();
+ List> expected =
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", BoundedTrieNode.TRUNCATED_TRUE),
+ Arrays.asList("a", "e", BoundedTrieNode.TRUNCATED_FALSE)));
+ assertEquals(expected, root.flattened());
+ }
+
+ @Test
+ public void testContains() {
+ BoundedTrieNode root = new BoundedTrieNode();
+ root.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "e"),
+ Arrays.asList("a", "b", "d"))));
+ assertTrue(root.contains(Arrays.asList("a", "b", "c")));
+ assertTrue(root.contains(Collections.singletonList("a")));
+ assertFalse(root.contains(Arrays.asList("a", "b", "f")));
+ assertFalse(root.contains(Collections.singletonList("z")));
+ }
+
+ @Test
+ public void testDeepCopy() {
+ BoundedTrieNode root = new BoundedTrieNode();
+ root.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "e"),
+ Arrays.asList("a", "b", "d"))));
+ BoundedTrieNode copy = root.deepCopy();
+ assertEquals(root, copy);
+
+ // Modify the original and ensure the copy is not affected
+ root.add(Arrays.asList("a", "f"));
+ assertNotEquals(root, copy);
+ }
+
+ @Test
+ public void testToProto() {
+ BoundedTrieNode root = new BoundedTrieNode();
+ root.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "e"),
+ Arrays.asList("a", "b", "d"))));
+ org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode proto = root.toProto();
+ BoundedTrieNode fromProto = BoundedTrieNode.fromProto(proto);
+ assertEquals(root, fromProto);
+ }
+
+ @Test
+ public void testFromProto() {
+ org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode.Builder builder =
+ org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode.newBuilder();
+ builder.putChildren(
+ "a",
+ org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode.newBuilder()
+ .putChildren(
+ "b",
+ org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode.newBuilder()
+ .putChildren(
+ "c",
+ org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrieNode.newBuilder()
+ .build())
+ .build())
+ .build());
+ BoundedTrieNode root = BoundedTrieNode.fromProto(builder.build());
+ assertEquals(1, root.getSize());
+ assertTrue(root.contains(Arrays.asList("a", "b", "c")));
+ assertFalse(root.isTruncated());
+ }
+
+ @Test
+ public void testBoundedTrieNodeEquals() {
+ BoundedTrieNode root1 = new BoundedTrieNode();
+ root1.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "e"),
+ Arrays.asList("a", "b", "d"))));
+
+ BoundedTrieNode root2 = new BoundedTrieNode();
+ root2.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "e"),
+ Arrays.asList("a", "b", "d"))));
+
+ assertEquals(root1, root2);
+ }
+
+ @Test
+ public void testBoundedTrieNodeHashCode() {
+ BoundedTrieNode root1 = new BoundedTrieNode();
+ root1.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "e"),
+ Arrays.asList("a", "b", "d"))));
+
+ BoundedTrieNode root2 = new BoundedTrieNode();
+ root2.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "e"),
+ Arrays.asList("a", "b", "d"))));
+
+ assertEquals(root1.hashCode(), root2.hashCode());
+ }
+
+ @Test
+ public void testBoundedTrieNodeToString() {
+ BoundedTrieNode root = new BoundedTrieNode();
+ root.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "e"),
+ Arrays.asList("a", "b", "d"))));
+ String expected = "{'abcfalse', 'abdfalse', 'aefalse'}";
+ assertEquals(expected, root.toString());
+ }
+
+ @Test
+ public void testEmptyTrie() {
+ BoundedTrieData trie = new BoundedTrieData();
+ assertEquals(0, trie.size());
+ assertTrue(trie.extractResult().getResult().isEmpty());
+ }
+
+ @Test
+ public void testSingleton() {
+ List path = ImmutableList.of("a", "b", "c");
+ BoundedTrieData trie = new BoundedTrieData(path);
+ assertEquals(1, trie.size());
+ assertEquals(
+ BoundedTrieResult.create(ImmutableSet.of(ImmutableList.of("a", "b", "c", "false"))),
+ trie.extractResult());
+ assertTrue(trie.contains(path));
+ assertFalse(trie.contains(ImmutableList.of("a", "b")));
+ }
+
+ @Test
+ public void testAddSingletonToTrie() {
+ BoundedTrieData trie = new BoundedTrieData(ImmutableList.of("a", "b"));
+ trie.add(ImmutableList.of("a", "c"));
+ assertEquals(2, trie.size());
+ assertEquals(
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ ImmutableList.of("a", "b", "false"), ImmutableList.of("a", "c", "false"))),
+ trie.extractResult());
+ }
+
+ @Test
+ public void testCombineEmptyTrie() {
+ BoundedTrieData trie1 = new BoundedTrieData();
+ BoundedTrieData trie2 = new BoundedTrieData();
+ trie2.add(ImmutableList.of("a", "b"));
+ trie1 = trie1.combine(trie2);
+ assertEquals(1, trie1.size());
+ assertEquals(
+ BoundedTrieResult.create(ImmutableSet.of(ImmutableList.of("a", "b", "false"))),
+ trie1.extractResult());
+ }
+
+ @Test
+ public void testCombineWithSingleton() {
+ BoundedTrieData trie1 = new BoundedTrieData();
+ trie1.add(ImmutableList.of("a", "b"));
+
+ BoundedTrieData trie2 = new BoundedTrieData(ImmutableList.of("c", "d"));
+
+ trie1 = trie1.combine(trie2);
+ assertEquals(2, trie1.size());
+ assertEquals(
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ ImmutableList.of("a", "b", "false"), ImmutableList.of("c", "d", "false"))),
+ trie1.extractResult());
+ }
+
+ @Test
+ public void testCombineWithItself() {
+ BoundedTrieData trie = new BoundedTrieData();
+ trie.add(ImmutableList.of("a", "b"));
+ trie = trie.combine(trie);
+ assertEquals(1, trie.size());
+ assertEquals(
+ BoundedTrieResult.create(ImmutableSet.of(ImmutableList.of("a", "b", "false"))),
+ trie.extractResult());
+ }
+
+ @Test
+ public void testClear() {
+ BoundedTrieData trie = new BoundedTrieData();
+ trie.add(ImmutableList.of("a", "b"));
+ trie.clear();
+ assertEquals(0, trie.size());
+ assertTrue(trie.extractResult().getResult().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")));
+ assertFalse(trie.contains(ImmutableList.of("a", "c")));
+ }
+
+ @Test
+ public void testEquals() {
+ BoundedTrieData trie1 = new BoundedTrieData();
+ trie1.add(ImmutableList.of("a", "b"));
+ BoundedTrieData trie2 = new BoundedTrieData();
+ trie2.add(ImmutableList.of("a", "b"));
+ assertEquals(trie1, trie2);
+ }
+
+ @Test
+ public void testHashCode() {
+ BoundedTrieData trie1 = new BoundedTrieData();
+ trie1.add(ImmutableList.of("a", "b"));
+ BoundedTrieData trie2 = new BoundedTrieData();
+ trie2.add(ImmutableList.of("a", "b"));
+ assertEquals(trie1.hashCode(), trie2.hashCode());
+ }
+
+ @Test
+ public void testToString() {
+ BoundedTrieData trie = new BoundedTrieData();
+ trie.add(ImmutableList.of("a", "b"));
+ assertTrue(trie.toString().contains("BoundedTrieData"));
+ }
+
+ @Test
+ public void testToProtoFromProtoEmpty() {
+ BoundedTrieData trie = new BoundedTrieData();
+ BoundedTrie proto = trie.toProto();
+ BoundedTrieData trieFromProto = BoundedTrieData.fromProto(proto);
+ assertEquals(trieFromProto, trie);
+ }
+
+ @Test
+ public void testToProtoFromProtoSingleton() {
+ BoundedTrieData trie = new BoundedTrieData(ImmutableList.of("a", "b"));
+ BoundedTrie proto = trie.toProto();
+ BoundedTrieData trieFromProto = BoundedTrieData.fromProto(proto);
+ assertEquals(trieFromProto, trie);
+ }
+
+ @Test
+ public void testToProtoFromProtoWithData() {
+ BoundedTrieData trie = new BoundedTrieData();
+ trie.add(ImmutableList.of("a", "b"));
+ trie.add(ImmutableList.of("a", "c"));
+ BoundedTrie proto = trie.toProto();
+ BoundedTrieData trieFromProto = BoundedTrieData.fromProto(proto);
+ assertEquals(trieFromProto, trie);
+ }
+
+ @Test
+ public void testConstructorInvalidInput() {
+ assertThrows(
+ AssertionError.class,
+ () -> new BoundedTrieData(ImmutableList.of("a"), new BoundedTrieNode(), 100));
+ }
+
+ @Test
+ public void testGetResultEmptyTrie() {
+ BoundedTrieData trie = new BoundedTrieData();
+ assertEquals(trie.extractResult(), BoundedTrieResult.empty());
+ }
+
+ @Test
+ public void testGetResultSingleton() {
+ List singletonList = ImmutableList.of("a", "b");
+ BoundedTrieData trie = new BoundedTrieData(singletonList);
+ BoundedTrieResult result = trie.extractResult();
+ assertEquals(
+ result, BoundedTrieResult.create(ImmutableSet.of(ImmutableList.of("a", "b", "false"))));
+ }
+
+ @Test
+ public void testGetCumulativeEmptyTrie() {
+ BoundedTrieData trie = new BoundedTrieData();
+ BoundedTrieData cumulativeTrie = trie.getCumulative();
+ assertEquals(cumulativeTrie, trie);
+ assertEquals(0, cumulativeTrie.size());
+ }
+
+ @Test
+ public void testGetCumulativeSingleton() {
+ List singletonList = ImmutableList.of("a", "b");
+ BoundedTrieData trie = new BoundedTrieData(singletonList);
+ BoundedTrieData cumulativeTrie = trie.getCumulative();
+ assertEquals(cumulativeTrie, trie);
+ assertNotSame(cumulativeTrie, trie);
+ // assert that the data in them are different
+ cumulativeTrie.add(ImmutableList.of("g", "h"));
+ assertTrue(cumulativeTrie.contains(ImmutableList.of("g", "h")));
+ assertFalse(trie.contains(ImmutableList.of("g", "h")));
+ assertEquals(1, trie.size());
+ }
+
+ @Test
+ public void testGetCumulativeWithRoot() {
+ BoundedTrieData trie = new BoundedTrieData();
+ trie.add(ImmutableList.of("a", "b"));
+ trie.add(ImmutableList.of("d", "e"));
+ BoundedTrieData cumulativeTrie = trie.getCumulative();
+ assertEquals(cumulativeTrie, trie);
+ assertNotSame(cumulativeTrie, trie);
+ // assert that the data in them are different
+ trie.add(ImmutableList.of("g", "h"));
+ assertTrue(trie.contains(ImmutableList.of("g", "h")));
+ 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()));
+ }
+}
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerImplTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerImplTest.java
index 5b3d71f4873e..69f7312f99ac 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerImplTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerImplTest.java
@@ -34,9 +34,14 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo;
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.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
import org.junit.Assert;
import org.junit.Test;
@@ -119,6 +124,38 @@ public void testCounterCumulatives() {
assertEquals(13L, (long) readC1.getCumulative());
}
+ @Test
+ public void testBoundedTrieCumulatives() {
+ MetricsContainerImpl container = new MetricsContainerImpl("step1");
+ BoundedTrieCell c1 = container.getBoundedTrie(MetricName.named("ns", "name1"));
+ BoundedTrieCell c2 = container.getBoundedTrie(MetricName.named("ns", "name2"));
+ c1.add("a");
+ c2.add("b");
+
+ container.getUpdates();
+ container.commitUpdates();
+ assertThat(
+ "Committing updates shouldn't affect cumulative counter values",
+ container.getCumulative().boundedTrieUpdates(),
+ containsInAnyOrder(
+ metricUpdate("name1", new BoundedTrieData(ImmutableList.of("a"))),
+ metricUpdate("name2", new BoundedTrieData(ImmutableList.of("b")))));
+
+ c1.add("c");
+ BoundedTrieData c1Expected = new BoundedTrieData(ImmutableList.of("a"));
+ c1Expected.add(ImmutableList.of("c"));
+ assertThat(
+ "Committing updates shouldn't affect cumulative counter values",
+ container.getCumulative().boundedTrieUpdates(),
+ containsInAnyOrder(
+ metricUpdate("name1", c1Expected),
+ metricUpdate("name2", new BoundedTrieData(ImmutableList.of("b")))));
+
+ BoundedTrieCell readC1 = container.tryGetBoundedTrie(MetricName.named("ns", "name1"));
+ assert readC1 != null;
+ assertEquals(c1Expected, readC1.getCumulative());
+ }
+
@Test
public void testDistributionDeltas() {
MetricsContainerImpl container = new MetricsContainerImpl("step1");
@@ -302,6 +339,38 @@ public void testMonitoringInfosArePopulatedForUserStringSets() {
assertThat(actualMonitoringInfos, containsInAnyOrder(builder1.build(), builder2.build()));
}
+ @Test
+ public void testMonitoringInfosArePopulatedForUserBoundedTries() {
+ MetricsContainerImpl testObject = new MetricsContainerImpl("step1");
+ BoundedTrieCell boundedTrieCellA = testObject.getBoundedTrie(MetricName.named("ns", "nameA"));
+ BoundedTrieCell boundedTrieCellB = testObject.getBoundedTrie(MetricName.named("ns", "nameB"));
+ boundedTrieCellA.add("A");
+ boundedTrieCellB.add("BBB");
+
+ SimpleMonitoringInfoBuilder builder1 = new SimpleMonitoringInfoBuilder();
+ builder1
+ .setUrn(MonitoringInfoConstants.Urns.USER_BOUNDED_TRIE)
+ .setLabel(MonitoringInfoConstants.Labels.NAMESPACE, "ns")
+ .setLabel(MonitoringInfoConstants.Labels.NAME, "nameA")
+ .setBoundedTrieValue(boundedTrieCellA.getCumulative())
+ .setLabel(MonitoringInfoConstants.Labels.PTRANSFORM, "step1");
+
+ SimpleMonitoringInfoBuilder builder2 = new SimpleMonitoringInfoBuilder();
+ builder2
+ .setUrn(MonitoringInfoConstants.Urns.USER_BOUNDED_TRIE)
+ .setLabel(MonitoringInfoConstants.Labels.NAMESPACE, "ns")
+ .setLabel(MonitoringInfoConstants.Labels.NAME, "nameB")
+ .setBoundedTrieValue(boundedTrieCellB.getCumulative())
+ .setLabel(MonitoringInfoConstants.Labels.PTRANSFORM, "step1");
+
+ List actualMonitoringInfos = new ArrayList<>();
+ for (MonitoringInfo mi : testObject.getMonitoringInfos()) {
+ actualMonitoringInfos.add(mi);
+ }
+
+ assertThat(actualMonitoringInfos, containsInAnyOrder(builder1.build(), builder2.build()));
+ }
+
@Test
public void testMonitoringInfosArePopulatedForSystemDistributions() {
MetricsContainerImpl testObject = new MetricsContainerImpl("step1");
@@ -457,6 +526,11 @@ public void testNotEquals() {
differentStringSets.getStringSet(MetricName.named("namespace", "name"));
Assert.assertNotEquals(metricsContainerImpl, differentStringSets);
Assert.assertNotEquals(metricsContainerImpl.hashCode(), differentStringSets.hashCode());
+
+ MetricsContainerImpl differentBoundedTrie = new MetricsContainerImpl("stepName");
+ differentBoundedTrie.getBoundedTrie(MetricName.named("namespace", "name"));
+ Assert.assertNotEquals(metricsContainerImpl, differentBoundedTrie);
+ Assert.assertNotEquals(metricsContainerImpl.hashCode(), differentBoundedTrie.hashCode());
}
@Test
@@ -491,4 +565,32 @@ public void testMatchMetric() {
Collections.singletonMap("name", "counter"));
assertFalse(MetricsContainerImpl.matchMetric(elementCountName, allowedMetricUrns));
}
+
+ @Test
+ public void testBoundedTrieMultithreaded() throws InterruptedException {
+ MetricsContainerImpl container = new MetricsContainerImpl("step1");
+ BoundedTrieCell boundedTrieCell =
+ container.getBoundedTrie(MetricName.named("test", "boundedTrie"));
+ int numThreads = 10;
+ int numUpdatesPerThread = 9; // be under the default bound of 100
+
+ CountDownLatch latch = new CountDownLatch(numThreads);
+ ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+ List tasks = new ArrayList<>();
+ for (int i = 0; i < numThreads; i++) {
+ tasks.add(
+ () -> {
+ for (int j = 0; j < numUpdatesPerThread; j++) {
+ boundedTrieCell.add("value-" + Thread.currentThread().getId() + "-" + j);
+ }
+ latch.countDown();
+ });
+ }
+
+ tasks.forEach(executor::execute);
+ latch.await(1, TimeUnit.MINUTES);
+ executor.shutdown();
+
+ assertEquals(numThreads * numUpdatesPerThread, boundedTrieCell.getCumulative().size());
+ }
}
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java
index 868c47f6a2e6..dcc3651e66fe 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java
@@ -29,6 +29,8 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo;
+import org.apache.beam.sdk.metrics.BoundedTrie;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Distribution;
import org.apache.beam.sdk.metrics.DistributionResult;
@@ -42,6 +44,7 @@
import org.apache.beam.sdk.metrics.MetricsFilter;
import org.apache.beam.sdk.metrics.StringSet;
import org.apache.beam.sdk.metrics.StringSetResult;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
import org.hamcrest.collection.IsIterableWithSize;
import org.joda.time.Instant;
@@ -62,8 +65,8 @@ public class MetricsContainerStepMapTest {
private static final String COUNTER_NAME = "myCounter";
private static final String DISTRIBUTION_NAME = "myDistribution";
private static final String GAUGE_NAME = "myGauge";
-
private static final String STRING_SET_NAME = "myStringSet";
+ private static final String BOUNDED_TRIE_NAME = "myBoundedTrie";
private static final long VALUE = 100;
@@ -78,6 +81,8 @@ public class MetricsContainerStepMapTest {
private static final StringSet stringSet =
Metrics.stringSet(MetricsContainerStepMapTest.class, STRING_SET_NAME);
+ private static final BoundedTrie boundedTrie =
+ Metrics.boundedTrie(MetricsContainerStepMapTest.class, BOUNDED_TRIE_NAME);
private static final MetricsContainerImpl metricsContainer;
@@ -89,6 +94,7 @@ public class MetricsContainerStepMapTest {
distribution.update(VALUE * 2);
gauge.set(VALUE);
stringSet.add(FIRST_STRING, SECOND_STRING);
+ boundedTrie.add(FIRST_STRING, SECOND_STRING);
} catch (IOException e) {
LOG.error(e.getMessage(), e);
}
@@ -112,6 +118,7 @@ public void testAttemptedAccumulatedMetricResults() {
assertIterableSize(step1res.getDistributions(), 1);
assertIterableSize(step1res.getGauges(), 1);
assertIterableSize(step1res.getStringSets(), 1);
+ assertIterableSize(step1res.getBoundedTries(), 1);
assertCounter(COUNTER_NAME, step1res, STEP1, VALUE, false);
assertDistribution(
@@ -129,6 +136,14 @@ public void testAttemptedAccumulatedMetricResults() {
StringSetResult.create(ImmutableSet.of(FIRST_STRING, SECOND_STRING)),
false);
+ assertBoundedTrie(
+ BOUNDED_TRIE_NAME,
+ step1res,
+ STEP1,
+ BoundedTrieResult.create(
+ ImmutableSet.of(ImmutableList.of(FIRST_STRING, SECOND_STRING, String.valueOf(false)))),
+ false);
+
MetricQueryResults step2res =
metricResults.queryMetrics(MetricsFilter.builder().addStep(STEP2).build());
@@ -136,6 +151,7 @@ public void testAttemptedAccumulatedMetricResults() {
assertIterableSize(step2res.getDistributions(), 1);
assertIterableSize(step2res.getGauges(), 1);
assertIterableSize(step2res.getStringSets(), 1);
+ assertIterableSize(step2res.getBoundedTries(), 1);
assertCounter(COUNTER_NAME, step2res, STEP2, VALUE * 2, false);
assertDistribution(
@@ -145,13 +161,19 @@ public void testAttemptedAccumulatedMetricResults() {
DistributionResult.create(VALUE * 6, 4, VALUE, VALUE * 2),
false);
assertGauge(GAUGE_NAME, step2res, STEP2, GaugeResult.create(VALUE, Instant.now()), false);
-
assertStringSet(
STRING_SET_NAME,
step2res,
STEP2,
StringSetResult.create(ImmutableSet.of(FIRST_STRING, SECOND_STRING)),
false);
+ assertBoundedTrie(
+ BOUNDED_TRIE_NAME,
+ step2res,
+ STEP2,
+ BoundedTrieResult.create(
+ ImmutableSet.of(ImmutableList.of(FIRST_STRING, SECOND_STRING, String.valueOf(false)))),
+ false);
MetricQueryResults allres = metricResults.allMetrics();
@@ -159,6 +181,7 @@ public void testAttemptedAccumulatedMetricResults() {
assertIterableSize(allres.getDistributions(), 2);
assertIterableSize(allres.getGauges(), 2);
assertIterableSize(allres.getStringSets(), 2);
+ assertIterableSize(allres.getBoundedTries(), 2);
}
@Test
@@ -222,6 +245,21 @@ public void testStringSetCommittedUnsupportedInAttemptedAccumulatedMetricResults
assertStringSet(STRING_SET_NAME, step1res, STEP1, StringSetResult.empty(), true);
}
+ @Test
+ public void testBoundedTrieCommittedUnsupportedInAttemptedAccumulatedMetricResults() {
+ MetricsContainerStepMap attemptedMetrics = new MetricsContainerStepMap();
+ attemptedMetrics.update(STEP1, metricsContainer);
+ MetricResults metricResults = asAttemptedOnlyMetricResults(attemptedMetrics);
+
+ MetricQueryResults step1res =
+ metricResults.queryMetrics(MetricsFilter.builder().addStep(STEP1).build());
+
+ thrown.expect(UnsupportedOperationException.class);
+ thrown.expectMessage("This runner does not currently support committed metrics results.");
+
+ assertBoundedTrie(BOUNDED_TRIE_NAME, step1res, STEP1, BoundedTrieResult.empty(), true);
+ }
+
@Test
public void testUserMetricDroppedOnUnbounded() {
MetricsContainerStepMap testObject = new MetricsContainerStepMap();
@@ -293,6 +331,7 @@ public void testAttemptedAndCommittedAccumulatedMetricResults() {
assertIterableSize(step1res.getDistributions(), 1);
assertIterableSize(step1res.getGauges(), 1);
assertIterableSize(step1res.getStringSets(), 1);
+ assertIterableSize(step1res.getBoundedTries(), 1);
assertCounter(COUNTER_NAME, step1res, STEP1, VALUE * 2, false);
assertDistribution(
@@ -308,6 +347,13 @@ public void testAttemptedAndCommittedAccumulatedMetricResults() {
STEP1,
StringSetResult.create(ImmutableSet.of(FIRST_STRING, SECOND_STRING)),
false);
+ assertBoundedTrie(
+ BOUNDED_TRIE_NAME,
+ step1res,
+ STEP1,
+ BoundedTrieResult.create(
+ ImmutableSet.of(ImmutableList.of(FIRST_STRING, SECOND_STRING, String.valueOf(false)))),
+ false);
assertCounter(COUNTER_NAME, step1res, STEP1, VALUE, true);
assertDistribution(
@@ -323,6 +369,13 @@ public void testAttemptedAndCommittedAccumulatedMetricResults() {
STEP1,
StringSetResult.create(ImmutableSet.of(FIRST_STRING, SECOND_STRING)),
true);
+ assertBoundedTrie(
+ BOUNDED_TRIE_NAME,
+ step1res,
+ STEP1,
+ BoundedTrieResult.create(
+ ImmutableSet.of(ImmutableList.of(FIRST_STRING, SECOND_STRING, String.valueOf(false)))),
+ true);
MetricQueryResults step2res =
metricResults.queryMetrics(MetricsFilter.builder().addStep(STEP2).build());
@@ -331,6 +384,7 @@ public void testAttemptedAndCommittedAccumulatedMetricResults() {
assertIterableSize(step2res.getDistributions(), 1);
assertIterableSize(step2res.getGauges(), 1);
assertIterableSize(step2res.getStringSets(), 1);
+ assertIterableSize(step2res.getBoundedTries(), 1);
assertCounter(COUNTER_NAME, step2res, STEP2, VALUE * 3, false);
assertDistribution(
@@ -346,6 +400,13 @@ public void testAttemptedAndCommittedAccumulatedMetricResults() {
STEP2,
StringSetResult.create(ImmutableSet.of(FIRST_STRING, SECOND_STRING)),
false);
+ assertBoundedTrie(
+ BOUNDED_TRIE_NAME,
+ step2res,
+ STEP2,
+ BoundedTrieResult.create(
+ ImmutableSet.of(ImmutableList.of(FIRST_STRING, SECOND_STRING, String.valueOf(false)))),
+ false);
assertCounter(COUNTER_NAME, step2res, STEP2, VALUE * 2, true);
assertDistribution(
@@ -361,11 +422,12 @@ public void testAttemptedAndCommittedAccumulatedMetricResults() {
STEP2,
StringSetResult.create(ImmutableSet.of(FIRST_STRING, SECOND_STRING)),
true);
- assertStringSet(
- STRING_SET_NAME,
+ assertBoundedTrie(
+ BOUNDED_TRIE_NAME,
step2res,
STEP2,
- StringSetResult.create(ImmutableSet.of(FIRST_STRING, SECOND_STRING)),
+ BoundedTrieResult.create(
+ ImmutableSet.of(ImmutableList.of(FIRST_STRING, SECOND_STRING, String.valueOf(false)))),
true);
MetricQueryResults allres = metricResults.queryMetrics(MetricsFilter.builder().build());
@@ -374,6 +436,7 @@ public void testAttemptedAndCommittedAccumulatedMetricResults() {
assertIterableSize(allres.getDistributions(), 2);
assertIterableSize(allres.getGauges(), 2);
assertIterableSize(allres.getStringSets(), 2);
+ assertIterableSize(allres.getBoundedTries(), 2);
}
@Test
@@ -428,6 +491,13 @@ public void testReset() {
STEP1,
StringSetResult.create(ImmutableSet.of(FIRST_STRING, SECOND_STRING)),
false);
+ assertBoundedTrie(
+ BOUNDED_TRIE_NAME,
+ allres,
+ STEP1,
+ BoundedTrieResult.create(
+ ImmutableSet.of(ImmutableList.of(FIRST_STRING, SECOND_STRING, String.valueOf(false)))),
+ false);
assertCounter(COUNTER_NAME, allres, STEP2, VALUE * 2, false);
assertDistribution(
@@ -443,6 +513,13 @@ public void testReset() {
STEP2,
StringSetResult.create(ImmutableSet.of(FIRST_STRING, SECOND_STRING)),
false);
+ assertBoundedTrie(
+ BOUNDED_TRIE_NAME,
+ allres,
+ STEP2,
+ BoundedTrieResult.create(
+ ImmutableSet.of(ImmutableList.of(FIRST_STRING, SECOND_STRING, String.valueOf(false)))),
+ false);
attemptedMetrics.reset();
metricResults = asAttemptedOnlyMetricResults(attemptedMetrics);
@@ -454,6 +531,7 @@ public void testReset() {
DISTRIBUTION_NAME, allres, STEP1, DistributionResult.IDENTITY_ELEMENT, false);
assertGauge(GAUGE_NAME, allres, STEP1, GaugeResult.empty(), false);
assertStringSet(STRING_SET_NAME, allres, STEP1, StringSetResult.empty(), false);
+ assertBoundedTrie(BOUNDED_TRIE_NAME, allres, STEP1, BoundedTrieResult.empty(), false);
// Check that the metrics container for STEP2 is reset
assertCounter(COUNTER_NAME, allres, STEP2, 0L, false);
@@ -461,6 +539,7 @@ public void testReset() {
DISTRIBUTION_NAME, allres, STEP2, DistributionResult.IDENTITY_ELEMENT, false);
assertGauge(GAUGE_NAME, allres, STEP2, GaugeResult.empty(), false);
assertStringSet(STRING_SET_NAME, allres, STEP2, StringSetResult.empty(), false);
+ assertBoundedTrie(BOUNDED_TRIE_NAME, allres, STEP2, BoundedTrieResult.empty(), false);
}
private void assertIterableSize(Iterable iterable, int size) {
@@ -510,4 +589,15 @@ private void assertStringSet(
metricQueryResults.getStringSets(),
hasItem(metricsResult(NAMESPACE, name, step, expected, isCommitted)));
}
+
+ private void assertBoundedTrie(
+ String name,
+ MetricQueryResults metricQueryResults,
+ String step,
+ BoundedTrieResult expected,
+ boolean isCommitted) {
+ assertThat(
+ metricQueryResults.getBoundedTries(),
+ hasItem(metricsResult(NAMESPACE, name, step, expected, isCommitted)));
+ }
}
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java
index 2d7ba61dbc95..25b634a87dc7 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java
@@ -18,12 +18,14 @@
package org.apache.beam.runners.core.metrics;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.HistogramParsingException;
+import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeBoundedTrie;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeDoubleCounter;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Counter;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Distribution;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Gauge;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Histogram;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeStringSet;
+import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeBoundedTrie;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeDoubleCounter;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeDoubleDistribution;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeInt64Counter;
@@ -33,10 +35,14 @@
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeStringSet;
import static org.junit.Assert.assertEquals;
+import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
+import org.apache.beam.runners.core.metrics.BoundedTrieData.BoundedTrieNode;
import org.apache.beam.sdk.testing.ExpectedLogs;
import org.apache.beam.sdk.util.HistogramData;
import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
import org.joda.time.Instant;
import org.junit.Rule;
@@ -105,6 +111,38 @@ public void testStringSetEncoding() {
assertEquals(data, decodeStringSet(payload));
}
+ @Test
+ public void testBoundedTrieEncoding() {
+ // test empty bounded trie encoding
+ BoundedTrieData data = new BoundedTrieData(Collections.emptyList());
+ ByteString payload = encodeBoundedTrie(data);
+ assertEquals(data, decodeBoundedTrie(payload));
+
+ // test singleton encoding
+ data = new BoundedTrieData(ImmutableList.of("ab"));
+ payload = encodeBoundedTrie(data);
+ assertEquals(data, decodeBoundedTrie(payload));
+
+ // test multiple element bounded trie encoding
+ data = new BoundedTrieData(ImmutableList.of("a", "b"));
+ data.add(ImmutableList.of("c", "d"));
+ payload = encodeBoundedTrie(data);
+ assertEquals(data, decodeBoundedTrie(payload));
+
+ // test encoding with trim
+ BoundedTrieNode root = new BoundedTrieNode();
+ root.addAll(
+ new ArrayList<>(
+ Arrays.asList(
+ Arrays.asList("a", "b", "c"),
+ Arrays.asList("a", "b", "d"),
+ Arrays.asList("a", "e"))));
+ root.trim();
+ data = new BoundedTrieData(root);
+ payload = encodeBoundedTrie(data);
+ assertEquals(data, decodeBoundedTrie(payload));
+ }
+
@Test
public void testInt64CounterEncoding() {
ByteString payload = encodeInt64Counter(1L);
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java
index b02c4f030b27..1cb6997c0710 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java
@@ -28,12 +28,14 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicReference;
import javax.annotation.concurrent.GuardedBy;
+import org.apache.beam.runners.core.metrics.BoundedTrieData;
import org.apache.beam.runners.core.metrics.DistributionData;
import org.apache.beam.runners.core.metrics.GaugeData;
import org.apache.beam.runners.core.metrics.MetricUpdates;
import org.apache.beam.runners.core.metrics.MetricUpdates.MetricUpdate;
import org.apache.beam.runners.core.metrics.MetricsMap;
import org.apache.beam.runners.core.metrics.StringSetData;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.GaugeResult;
import org.apache.beam.sdk.metrics.MetricFiltering;
@@ -239,6 +241,28 @@ public StringSetResult extract(StringSetData data) {
}
};
+ private static final MetricAggregation BOUNDED_TRIE =
+ new MetricAggregation() {
+ @Override
+ public BoundedTrieData zero() {
+ return new BoundedTrieData();
+ }
+
+ @Override
+ public BoundedTrieData combine(Iterable updates) {
+ BoundedTrieData result = new BoundedTrieData();
+ for (BoundedTrieData update : updates) {
+ result = result.combine(update);
+ }
+ return result;
+ }
+
+ @Override
+ public BoundedTrieResult extract(BoundedTrieData data) {
+ return data.extractResult();
+ }
+ };
+
/** The current values of counters in memory. */
private final MetricsMap> counters;
@@ -247,6 +271,7 @@ public StringSetResult extract(StringSetData data) {
private final MetricsMap> gauges;
private final MetricsMap> stringSet;
+ private final MetricsMap> boundedTrie;
DirectMetrics(ExecutorService executorService) {
this.counters = new MetricsMap<>(unusedKey -> new DirectMetric<>(COUNTER, executorService));
@@ -254,6 +279,8 @@ public StringSetResult extract(StringSetData data) {
new MetricsMap<>(unusedKey -> new DirectMetric<>(DISTRIBUTION, executorService));
this.gauges = new MetricsMap<>(unusedKey -> new DirectMetric<>(GAUGE, executorService));
this.stringSet = new MetricsMap<>(unusedKey -> new DirectMetric<>(STRING_SET, executorService));
+ this.boundedTrie =
+ new MetricsMap<>(unusedKey -> new DirectMetric<>(BOUNDED_TRIE, executorService));
}
@Override
@@ -279,11 +306,19 @@ public MetricQueryResults queryMetrics(@Nullable MetricsFilter filter) {
maybeExtractResult(filter, stringSetResult, stringSet);
}
+ ImmutableList.Builder> boundedTrieResult =
+ ImmutableList.builder();
+ for (Entry> boundedTrie :
+ boundedTrie.entries()) {
+ maybeExtractResult(filter, boundedTrieResult, boundedTrie);
+ }
+
return MetricQueryResults.create(
counterResults.build(),
distributionResults.build(),
gaugeResults.build(),
- stringSetResult.build());
+ stringSetResult.build(),
+ boundedTrieResult.build());
}
private void maybeExtractResult(
@@ -310,10 +345,12 @@ public void updatePhysical(CommittedBundle> bundle, MetricUpdates updates) {
for (MetricUpdate gauge : updates.gaugeUpdates()) {
gauges.get(gauge.getKey()).updatePhysical(bundle, gauge.getUpdate());
}
-
for (MetricUpdate sSet : updates.stringSetUpdates()) {
stringSet.get(sSet.getKey()).updatePhysical(bundle, sSet.getUpdate());
}
+ for (MetricUpdate bTrie : updates.boundedTrieUpdates()) {
+ boundedTrie.get(bTrie.getKey()).updatePhysical(bundle, bTrie.getUpdate());
+ }
}
public void commitPhysical(CommittedBundle> bundle, MetricUpdates updates) {
@@ -329,6 +366,9 @@ public void commitPhysical(CommittedBundle> bundle, MetricUpdates updates) {
for (MetricUpdate sSet : updates.stringSetUpdates()) {
stringSet.get(sSet.getKey()).commitPhysical(bundle, sSet.getUpdate());
}
+ for (MetricUpdate bTrie : updates.boundedTrieUpdates()) {
+ boundedTrie.get(bTrie.getKey()).commitPhysical(bundle, bTrie.getUpdate());
+ }
}
/** Apply metric updates that represent new logical values from a bundle being committed. */
@@ -345,5 +385,8 @@ public void commitLogical(CommittedBundle> bundle, MetricUpdates updates) {
for (MetricUpdate sSet : updates.stringSetUpdates()) {
stringSet.get(sSet.getKey()).commitLogical(bundle, sSet.getUpdate());
}
+ for (MetricUpdate bTrie : updates.boundedTrieUpdates()) {
+ boundedTrie.get(bTrie.getKey()).commitLogical(bundle, bTrie.getUpdate());
+ }
}
}
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java
index 00df20c4ac39..b479b73a92cd 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java
@@ -26,11 +26,13 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import org.apache.beam.runners.core.metrics.BoundedTrieData;
import org.apache.beam.runners.core.metrics.DistributionData;
import org.apache.beam.runners.core.metrics.GaugeData;
import org.apache.beam.runners.core.metrics.MetricUpdates;
import org.apache.beam.runners.core.metrics.MetricUpdates.MetricUpdate;
import org.apache.beam.runners.core.metrics.StringSetData;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.GaugeResult;
import org.apache.beam.sdk.metrics.MetricKey;
@@ -89,10 +91,13 @@ public void testApplyCommittedNoFilter() {
MetricKey.create("step1", NAME1), DistributionData.create(8, 2, 3, 5))),
ImmutableList.of(
MetricUpdate.create(MetricKey.create("step1", NAME4), GaugeData.create(15L))),
+ ImmutableList.of(
+ MetricUpdate.create(
+ MetricKey.create("step1", NAME4), StringSetData.create(ImmutableSet.of("ab")))),
ImmutableList.of(
MetricUpdate.create(
MetricKey.create("step1", NAME4),
- StringSetData.create(ImmutableSet.of("ab"))))));
+ new BoundedTrieData(ImmutableList.of("ab"))))));
metrics.commitLogical(
bundle1,
MetricUpdates.create(
@@ -104,10 +109,13 @@ public void testApplyCommittedNoFilter() {
MetricKey.create("step1", NAME1), DistributionData.create(4, 1, 4, 4))),
ImmutableList.of(
MetricUpdate.create(MetricKey.create("step1", NAME4), GaugeData.create(27L))),
+ ImmutableList.of(
+ MetricUpdate.create(
+ MetricKey.create("step1", NAME4), StringSetData.create(ImmutableSet.of("cd")))),
ImmutableList.of(
MetricUpdate.create(
MetricKey.create("step1", NAME4),
- StringSetData.create(ImmutableSet.of("cd"))))));
+ new BoundedTrieData(ImmutableList.of("cd"))))));
MetricQueryResults results = metrics.allMetrics();
assertThat(
@@ -144,6 +152,17 @@ public void testApplyCommittedNoFilter() {
contains(
committedMetricsResult(
"ns2", "name2", "step1", StringSetResult.create(ImmutableSet.of("ab", "cd")))));
+ assertThat(
+ results.getBoundedTries(),
+ contains(
+ committedMetricsResult(
+ "ns2",
+ "name2",
+ "step1",
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ ImmutableList.of("ab", String.valueOf(false)),
+ ImmutableList.of("cd", String.valueOf(false)))))));
}
@SuppressWarnings("unchecked")
@@ -157,6 +176,7 @@ public void testApplyAttemptedCountersQueryOneNamespace() {
MetricUpdate.create(MetricKey.create("step1", NAME3), 8L)),
ImmutableList.of(),
ImmutableList.of(),
+ ImmutableList.of(),
ImmutableList.of()));
metrics.updatePhysical(
bundle1,
@@ -166,6 +186,7 @@ public void testApplyAttemptedCountersQueryOneNamespace() {
MetricUpdate.create(MetricKey.create("step1", NAME3), 4L)),
ImmutableList.of(),
ImmutableList.of(),
+ ImmutableList.of(),
ImmutableList.of()));
MetricQueryResults results =
@@ -195,6 +216,7 @@ public void testApplyAttemptedQueryCompositeScope() {
MetricUpdate.create(MetricKey.create("Outer1/Inner2", NAME1), 8L)),
ImmutableList.of(),
ImmutableList.of(),
+ ImmutableList.of(),
ImmutableList.of()));
metrics.updatePhysical(
bundle1,
@@ -204,6 +226,7 @@ public void testApplyAttemptedQueryCompositeScope() {
MetricUpdate.create(MetricKey.create("Outer2/Inner2", NAME1), 18L)),
ImmutableList.of(),
ImmutableList.of(),
+ ImmutableList.of(),
ImmutableList.of()));
MetricQueryResults results =
@@ -233,6 +256,7 @@ public void testPartialScopeMatchingInMetricsQuery() {
MetricUpdate.create(MetricKey.create("Top1/Outer1/Inner2", NAME1), 8L)),
ImmutableList.of(),
ImmutableList.of(),
+ ImmutableList.of(),
ImmutableList.of()));
metrics.updatePhysical(
bundle1,
@@ -242,6 +266,7 @@ public void testPartialScopeMatchingInMetricsQuery() {
MetricUpdate.create(MetricKey.create("Top1/Outer2/Inner2", NAME1), 18L)),
ImmutableList.of(),
ImmutableList.of(),
+ ImmutableList.of(),
ImmutableList.of()));
MetricQueryResults results =
diff --git a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/CustomMetricQueryResults.java b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/CustomMetricQueryResults.java
index 96c0374067cf..44a4e80c5f44 100644
--- a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/CustomMetricQueryResults.java
+++ b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/CustomMetricQueryResults.java
@@ -19,6 +19,7 @@
import java.util.Collections;
import java.util.List;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.GaugeResult;
import org.apache.beam.sdk.metrics.MetricKey;
@@ -27,6 +28,7 @@
import org.apache.beam.sdk.metrics.MetricResult;
import org.apache.beam.sdk.metrics.MetricsSink;
import org.apache.beam.sdk.metrics.StringSetResult;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
import org.joda.time.Instant;
@@ -82,4 +84,13 @@ public Iterable> getStringSets() {
StringSetResult.create(ImmutableSet.of("ab")),
StringSetResult.create(ImmutableSet.of("cd")));
}
+
+ @Override
+ public Iterable> getBoundedTries() {
+ return makeResults(
+ "s3",
+ "n3",
+ BoundedTrieResult.create(ImmutableSet.of(ImmutableList.of("ab", String.valueOf(false)))),
+ BoundedTrieResult.create(ImmutableSet.of(ImmutableList.of("cd", String.valueOf(false)))));
+ }
}
diff --git a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSinkTest.java b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSinkTest.java
index 10e9481d271b..d9a49c67111d 100644
--- a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSinkTest.java
+++ b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSinkTest.java
@@ -87,7 +87,10 @@ public void testWriteMetricsWithCommittedSupported() throws Exception {
metricsHttpSink.writeMetrics(metricQueryResults);
countDownLatch.await();
String expected =
- "{\"counters\":[{\"attempted\":20,\"committed\":10,\"name\":{\"name\":\"n1\","
+ "{\"boundedTries\":[{\"attempted\":{\"result\":[[\"cd\",\"false\"]]},"
+ + "\"committed\":{\"result\":[[\"ab\",\"false\"]]},\"name\":{\"name\":\"n3\","
+ + "\"namespace\":\"ns1\"},\"step\":\"s3\"}],"
+ + "\"counters\":[{\"attempted\":20,\"committed\":10,\"name\":{\"name\":\"n1\","
+ "\"namespace\":\"ns1\"},\"step\":\"s1\"}],\"distributions\":[{\"attempted\":"
+ "{\"count\":4,\"max\":9,\"mean\":6.25,\"min\":3,\"sum\":25},\"committed\":"
+ "{\"count\":2,\"max\":8,\"mean\":5.0,\"min\":5,\"sum\":10},\"name\":{\"name\":\"n2\","
@@ -111,7 +114,9 @@ public void testWriteMetricsWithCommittedUnSupported() throws Exception {
metricsHttpSink.writeMetrics(metricQueryResults);
countDownLatch.await();
String expected =
- "{\"counters\":[{\"attempted\":20,\"name\":{\"name\":\"n1\","
+ "{\"boundedTries\":[{\"attempted\":{\"result\":[[\"cd\",\"false\"]]},"
+ + "\"name\":{\"name\":\"n3\",\"namespace\":\"ns1\"},\"step\":\"s3\"}],"
+ + "\"counters\":[{\"attempted\":20,\"name\":{\"name\":\"n1\","
+ "\"namespace\":\"ns1\"},\"step\":\"s1\"}],\"distributions\":[{\"attempted\":"
+ "{\"count\":4,\"max\":9,\"mean\":6.25,\"min\":3,\"sum\":25},\"name\":{\"name\":\"n2\""
+ ",\"namespace\":\"ns1\"},\"step\":\"s2\"}],\"gauges\":[{\"attempted\":{\"timestamp\":"
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java
index 46fdce507c3d..c79897b5e5c3 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java
@@ -30,6 +30,7 @@
import java.util.HashSet;
import java.util.List;
import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.GaugeResult;
import org.apache.beam.sdk.metrics.MetricFiltering;
@@ -103,12 +104,13 @@ public MetricQueryResults queryMetrics(MetricsFilter filter) {
ImmutableList> distributions = ImmutableList.of();
ImmutableList> gauges = ImmutableList.of();
ImmutableList> stringSets = ImmutableList.of();
+ ImmutableList> boudedTries = 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);
+ return MetricQueryResults.create(counters, distributions, gauges, stringSets, boudedTries);
}
metricUpdates = firstNonNull(jobMetrics.getMetrics(), Collections.emptyList());
return populateMetricQueryResults(metricUpdates, filter);
@@ -386,7 +388,8 @@ public MetricQueryResults build() {
extractor.getCounterResults(),
extractor.getDistributionResults(),
extractor.getGaugeResults(),
- extractor.geStringSetResults());
+ extractor.geStringSetResults(),
+ ImmutableList.of());
}
}
}
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowMetricsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowMetricsTest.java
index 9b8e3cc871da..745b065ea841 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowMetricsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowMetricsTest.java
@@ -119,6 +119,7 @@ public void testEmptyMetricUpdates() throws IOException {
assertThat(ImmutableList.copyOf(result.getCounters()), is(empty()));
assertThat(ImmutableList.copyOf(result.getDistributions()), is(empty()));
assertThat(ImmutableList.copyOf(result.getStringSets()), is(empty()));
+ assertThat(ImmutableList.copyOf(result.getBoundedTries()), is(empty()));
}
@Test
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java
index f9cd098edaa6..8966cfa9f7fa 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java
@@ -19,6 +19,7 @@
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import org.apache.beam.runners.core.metrics.ExecutionStateTracker;
+import org.apache.beam.sdk.metrics.BoundedTrie;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Distribution;
import org.apache.beam.sdk.metrics.Gauge;
@@ -79,6 +80,11 @@ public StringSet getStringSet(MetricName metricName) {
return getCurrentContainer().getStringSet(metricName);
}
+ @Override
+ public BoundedTrie getBoundedTrie(MetricName metricName) {
+ return getCurrentContainer().getBoundedTrie(metricName);
+ }
+
@Override
public Histogram getPerWorkerHistogram(
MetricName metricName, HistogramData.BucketType bucketType) {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java
index 7cc0dc68f7e7..7c5aadefb9ac 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java
@@ -30,10 +30,12 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import javax.annotation.Nonnull;
+import org.apache.beam.runners.core.metrics.BoundedTrieCell;
import org.apache.beam.runners.core.metrics.DistributionData;
import org.apache.beam.runners.core.metrics.GaugeCell;
import org.apache.beam.runners.core.metrics.MetricsMap;
import org.apache.beam.runners.core.metrics.StringSetCell;
+import org.apache.beam.sdk.metrics.BoundedTrie;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Distribution;
import org.apache.beam.sdk.metrics.Gauge;
@@ -74,6 +76,9 @@ public class StreamingStepMetricsContainer implements MetricsContainer {
private MetricsMap distributions =
new MetricsMap<>(DeltaDistributionCell::new);
+ private MetricsMap boundedTries =
+ new MetricsMap<>(BoundedTrieCell::new);
+
private final ConcurrentHashMap perWorkerHistograms =
new ConcurrentHashMap<>();
@@ -168,6 +173,11 @@ public StringSet getStringSet(MetricName metricName) {
return stringSet.get(metricName);
}
+ @Override
+ public BoundedTrie getBoundedTrie(MetricName metricName) {
+ return boundedTries.get(metricName);
+ }
+
@Override
public Histogram getPerWorkerHistogram(
MetricName metricName, HistogramData.BucketType bucketType) {
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/FailedRunningPipelineResults.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/FailedRunningPipelineResults.java
index 67cf3280a83c..d170f02be2d2 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/FailedRunningPipelineResults.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/FailedRunningPipelineResults.java
@@ -19,6 +19,7 @@
import java.util.Collections;
import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.GaugeResult;
import org.apache.beam.sdk.metrics.MetricQueryResults;
@@ -90,6 +91,11 @@ public Iterable> getGauges() {
public Iterable> getStringSets() {
return Collections.emptyList();
}
+
+ @Override
+ public Iterable> getBoundedTries() {
+ return Collections.emptyList();
+ }
};
}
};
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/BoundedTrieImpl.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/BoundedTrieImpl.java
new file mode 100644
index 000000000000..bd7a30ac1745
--- /dev/null
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/BoundedTrieImpl.java
@@ -0,0 +1,48 @@
+/*
+ * 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.runners.jet.metrics;
+
+import java.util.Arrays;
+import org.apache.beam.runners.core.metrics.BoundedTrieData;
+import org.apache.beam.sdk.metrics.BoundedTrie;
+import org.apache.beam.sdk.metrics.MetricName;
+
+/** Implementation of {@link BoundedTrie}. */
+public class BoundedTrieImpl extends AbstractMetric implements BoundedTrie {
+
+ private final BoundedTrieData boundedTrieDataData = new BoundedTrieData();
+
+ public BoundedTrieImpl(MetricName name) {
+ super(name);
+ }
+
+ @Override
+ BoundedTrieData getValue() {
+ return boundedTrieDataData;
+ }
+
+ @Override
+ public void add(Iterable values) {
+ boundedTrieDataData.add(values);
+ }
+
+ @Override
+ public void add(String ... values) {
+ add(Arrays.asList(values));
+ }
+}
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricResults.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricResults.java
index 44681a626cc0..5277246b2a3f 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricResults.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricResults.java
@@ -21,11 +21,13 @@
import java.util.HashMap;
import java.util.Map;
import javax.annotation.concurrent.GuardedBy;
+import org.apache.beam.runners.core.metrics.BoundedTrieData;
import org.apache.beam.runners.core.metrics.DistributionData;
import org.apache.beam.runners.core.metrics.GaugeData;
import org.apache.beam.runners.core.metrics.MetricUpdates;
import org.apache.beam.runners.core.metrics.MetricUpdates.MetricUpdate;
import org.apache.beam.runners.core.metrics.StringSetData;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.GaugeResult;
import org.apache.beam.sdk.metrics.MetricFiltering;
@@ -57,6 +59,9 @@ public class JetMetricResults extends MetricResults {
@GuardedBy("this")
private final StringSets stringSet = new StringSets();
+ @GuardedBy("this")
+ private final BoundedTries boundedTries = new BoundedTries();
+
@GuardedBy("this")
private IMap metricsAccumulator;
@@ -78,7 +83,8 @@ public synchronized MetricQueryResults queryMetrics(@Nullable MetricsFilter filt
counters.filter(filter),
distributions.filter(filter),
gauges.filter(filter),
- stringSet.filter(filter));
+ stringSet.filter(filter),
+ boundedTries.filter(filter));
}
private synchronized void updateLocalMetrics(IMap metricsAccumulator) {
@@ -104,16 +110,19 @@ private static class QueryResults extends MetricQueryResults {
private final Iterable> distributions;
private final Iterable> gauges;
private final Iterable> stringSets;
+ private final Iterable> boundedTries;
private QueryResults(
Iterable> counters,
Iterable> distributions,
Iterable> gauges,
- Iterable> stringSets) {
+ Iterable> stringSets,
+ Iterable> boundedTries) {
this.counters = counters;
this.distributions = distributions;
this.gauges = gauges;
this.stringSets = stringSets;
+ this.boundedTries = boundedTries;
}
@Override
@@ -135,6 +144,11 @@ public Iterable> getGauges() {
public Iterable> getStringSets() {
return stringSets;
}
+
+ @Override
+ public Iterable> getBoundedTries() {
+ return boundedTries;
+ }
}
private static class Counters {
@@ -262,4 +276,36 @@ private MetricResult toUpdateResult(
return MetricResult.create(key, stringSetResult, stringSetResult);
}
}
+
+ private static class BoundedTries {
+
+ private final Map boundedTries = new HashMap<>();
+
+ void merge(Iterable> updates) {
+ for (MetricUpdate update : updates) {
+ MetricKey key = update.getKey();
+ BoundedTrieData oldStringSet = boundedTries.getOrDefault(key, new BoundedTrieData());
+ BoundedTrieData updatedStringSet = update.getUpdate().combine(oldStringSet);
+ boundedTries.put(key, updatedStringSet);
+ }
+ }
+
+ void clear() {
+ boundedTries.clear();
+ }
+
+ Iterable> filter(MetricsFilter filter) {
+ return FluentIterable.from(boundedTries.entrySet())
+ .filter(matchesFilter(filter))
+ .transform(this::toUpdateResult)
+ .toList();
+ }
+
+ private MetricResult toUpdateResult(
+ Map.Entry entry) {
+ MetricKey key = entry.getKey();
+ BoundedTrieResult boundedTrieResult = entry.getValue().extractResult();
+ return MetricResult.create(key, boundedTrieResult, boundedTrieResult);
+ }
+ }
}
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricsContainer.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricsContainer.java
index 64455d704c9b..c9a6fcc09292 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricsContainer.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricsContainer.java
@@ -23,10 +23,12 @@
import java.io.Serializable;
import java.util.HashMap;
import java.util.Map;
+import org.apache.beam.runners.core.metrics.BoundedTrieData;
import org.apache.beam.runners.core.metrics.DistributionData;
import org.apache.beam.runners.core.metrics.GaugeData;
import org.apache.beam.runners.core.metrics.MetricUpdates;
import org.apache.beam.runners.core.metrics.StringSetData;
+import org.apache.beam.sdk.metrics.BoundedTrie;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Distribution;
import org.apache.beam.sdk.metrics.Gauge;
@@ -50,6 +52,7 @@ public static String getMetricsMapName(long jobId) {
private final Map distributions = new HashMap<>();
private final Map gauges = new HashMap<>();
private final Map stringSets = new HashMap<>();
+ private final Map boundedTries = new HashMap<>();
private final IMap accumulator;
@@ -79,9 +82,15 @@ public StringSet getStringSet(MetricName metricName) {
return stringSets.computeIfAbsent(metricName, StringSetImpl::new);
}
+ @Override
+ public BoundedTrie getBoundedTrie(MetricName metricName) {
+ return boundedTries.computeIfAbsent(metricName, BoundedTrieImpl::new);
+ }
+
@SuppressWarnings("FutureReturnValueIgnored")
public void flush(boolean async) {
- if (counters.isEmpty() && distributions.isEmpty() && gauges.isEmpty() && stringSets.isEmpty()) {
+ if (counters.isEmpty() && distributions.isEmpty() && gauges.isEmpty() &&
+ stringSets.isEmpty() && boundedTries.isEmpty()) {
return;
}
@@ -91,7 +100,9 @@ public void flush(boolean async) {
ImmutableList> gauges = extractUpdates(this.gauges);
ImmutableList> stringSets =
extractUpdates(this.stringSets);
- MetricUpdates updates = new MetricUpdatesImpl(counters, distributions, gauges, stringSets);
+ ImmutableList> boundedTries =
+ extractUpdates(this.boundedTries);
+ MetricUpdates updates = new MetricUpdatesImpl(counters, distributions, gauges, stringSets, boundedTries);
if (async) {
accumulator.setAsync(metricsKey, updates);
@@ -121,16 +132,19 @@ private static class MetricUpdatesImpl extends MetricUpdates implements Serializ
private final Iterable> distributions;
private final Iterable> gauges;
private final Iterable> stringSets;
+ private final Iterable> boundedTries;
MetricUpdatesImpl(
Iterable> counters,
Iterable> distributions,
Iterable> gauges,
- Iterable> stringSets) {
+ Iterable> stringSets,
+ Iterable> boundedTries) {
this.counters = counters;
this.distributions = distributions;
this.gauges = gauges;
this.stringSets = stringSets;
+ this.boundedTries = boundedTries;
}
@Override
@@ -152,5 +166,10 @@ public Iterable> gaugeUpdates() {
public Iterable> stringSetUpdates() {
return stringSets;
}
+
+ @Override
+ public Iterable> boundedTrieUpdates() {
+ return boundedTries;
+ }
}
}
diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableMetrics.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableMetrics.java
index 7ae57a4b3089..66a3b6bfac70 100644
--- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableMetrics.java
+++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableMetrics.java
@@ -17,10 +17,12 @@
*/
package org.apache.beam.runners.portability;
+import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.BOUNDED_TRIE_TYPE;
import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.DISTRIBUTION_INT64_TYPE;
import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.LATEST_INT64_TYPE;
import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.SET_STRING_TYPE;
import static org.apache.beam.runners.core.metrics.MonitoringInfoConstants.TypeUrns.SUM_INT64_TYPE;
+import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeBoundedTrie;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Counter;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Distribution;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.decodeInt64Gauge;
@@ -32,9 +34,11 @@
import java.util.stream.Collectors;
import org.apache.beam.model.jobmanagement.v1.JobApi;
import org.apache.beam.model.pipeline.v1.MetricsApi;
+import org.apache.beam.runners.core.metrics.BoundedTrieData;
import org.apache.beam.runners.core.metrics.DistributionData;
import org.apache.beam.runners.core.metrics.GaugeData;
import org.apache.beam.runners.core.metrics.StringSetData;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.GaugeResult;
import org.apache.beam.sdk.metrics.MetricFiltering;
@@ -58,16 +62,19 @@ public class PortableMetrics extends MetricResults {
private Iterable> distributions;
private Iterable> gauges;
private Iterable> stringSets;
+ private Iterable> boundedTries;
private PortableMetrics(
Iterable> counters,
Iterable> distributions,
Iterable> gauges,
- Iterable> stringSets) {
+ Iterable> stringSets,
+ Iterable> boundedTries) {
this.counters = counters;
this.distributions = distributions;
this.gauges = gauges;
this.stringSets = stringSets;
+ this.boundedTries = boundedTries;
}
public static PortableMetrics of(JobApi.MetricResults jobMetrics) {
@@ -84,7 +91,10 @@ public MetricQueryResults queryMetrics(MetricsFilter filter) {
(distribution) -> MetricFiltering.matches(filter, distribution.getKey())),
Iterables.filter(this.gauges, (gauge) -> MetricFiltering.matches(filter, gauge.getKey())),
Iterables.filter(
- this.stringSets, (stringSet) -> MetricFiltering.matches(filter, stringSet.getKey())));
+ this.stringSets, (stringSet) -> MetricFiltering.matches(filter, stringSet.getKey())),
+ Iterables.filter(
+ this.boundedTries,
+ (boundedTries) -> MetricFiltering.matches(filter, boundedTries.getKey())));
}
private static PortableMetrics convertMonitoringInfosToMetricResults(
@@ -101,8 +111,14 @@ private static PortableMetrics convertMonitoringInfosToMetricResults(
extractGaugeMetricsFromJobMetrics(monitoringInfoList);
Iterable> stringSetFromMetrics =
extractStringSetMetricsFromJobMetrics(monitoringInfoList);
+ Iterable> boundedTrieFromMetrics =
+ extractBoundedTrieMetricsFromJobMetrics(monitoringInfoList);
return new PortableMetrics(
- countersFromJobMetrics, distributionsFromMetrics, gaugesFromMetrics, stringSetFromMetrics);
+ countersFromJobMetrics,
+ distributionsFromMetrics,
+ gaugesFromMetrics,
+ stringSetFromMetrics,
+ boundedTrieFromMetrics);
}
private static Iterable>
@@ -145,6 +161,15 @@ private static Iterable> extractStringSetMetricsFr
.collect(Collectors.toList());
}
+ private static Iterable> extractBoundedTrieMetricsFromJobMetrics(
+ List monitoringInfoList) {
+ return monitoringInfoList.stream()
+ .filter(item -> BOUNDED_TRIE_TYPE.equals(item.getType()))
+ .filter(item -> item.getLabelsMap().get(NAMESPACE_LABEL) != null)
+ .map(PortableMetrics::convertBoundedTrieMonitoringInfoToBoundedTrie)
+ .collect(Collectors.toList());
+ }
+
private static MetricResult convertStringSetMonitoringInfoToStringSet(
MetricsApi.MonitoringInfo monitoringInfo) {
Map labelsMap = monitoringInfo.getLabelsMap();
@@ -158,6 +183,19 @@ private static MetricResult convertStringSetMonitoringInfoToStr
return MetricResult.create(key, false, result);
}
+ private static MetricResult convertBoundedTrieMonitoringInfoToBoundedTrie(
+ MetricsApi.MonitoringInfo monitoringInfo) {
+ Map labelsMap = monitoringInfo.getLabelsMap();
+ MetricKey key =
+ MetricKey.create(
+ labelsMap.get(STEP_NAME_LABEL),
+ MetricName.named(labelsMap.get(NAMESPACE_LABEL), labelsMap.get(METRIC_NAME_LABEL)));
+
+ BoundedTrieData data = decodeBoundedTrie(monitoringInfo.getPayload());
+ BoundedTrieResult result = BoundedTrieResult.create(data.extractResult().getResult());
+ return MetricResult.create(key, false, result);
+ }
+
private static MetricResult convertDistributionMonitoringInfoToDistribution(
MetricsApi.MonitoringInfo monitoringInfo) {
Map labelsMap = monitoringInfo.getLabelsMap();
diff --git a/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java b/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java
index 788d4a43319d..32404663aef7 100644
--- a/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java
+++ b/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java
@@ -17,6 +17,7 @@
*/
package org.apache.beam.runners.portability;
+import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeBoundedTrie;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeInt64Counter;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeInt64Distribution;
import static org.apache.beam.runners.core.metrics.MonitoringInfoEncodings.encodeInt64Gauge;
@@ -34,6 +35,7 @@
import org.apache.beam.model.jobmanagement.v1.JobApi.JobState;
import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
import org.apache.beam.model.pipeline.v1.MetricsApi;
+import org.apache.beam.runners.core.metrics.BoundedTrieData;
import org.apache.beam.runners.core.metrics.DistributionData;
import org.apache.beam.runners.core.metrics.GaugeData;
import org.apache.beam.runners.core.metrics.StringSetData;
@@ -52,6 +54,7 @@
import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.Server;
import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.inprocess.InProcessServerBuilder;
import org.apache.beam.vendor.grpc.v1p60p1.io.grpc.testing.GrpcCleanupRule;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
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.ImmutableSet;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.ByteStreams;
@@ -73,6 +76,7 @@ public class PortableRunnerTest implements Serializable {
private static final String DIST_TYPE = "beam:metrics:distribution_int64:v1";
private static final String GAUGE_TYPE = "beam:metrics:latest_int64:v1";
private static final String STRING_SET_TYPE = "beam:metrics:set_string:v1";
+ private static final String BOUNDED_TRIE_TYPE = "beam:metrics:bounded_trie:v1";
private static final String NAMESPACE_LABEL = "NAMESPACE";
private static final String METRIC_NAME_LABEL = "NAME";
private static final String STEP_NAME_LABEL = "PTRANSFORM";
@@ -133,6 +137,9 @@ public void extractsMetrics() throws Exception {
assertThat(
metricQueryResults.getStringSets().iterator().next().getAttempted().getStringSet(),
is(STRING_SET_VALUE));
+ assertThat(
+ metricQueryResults.getBoundedTries().iterator().next().getAttempted().getResult(),
+ is(ImmutableSet.of(ImmutableList.of("ab", String.valueOf(false)))));
}
private JobApi.MetricResults generateMetricResults() throws Exception {
@@ -171,11 +178,19 @@ private JobApi.MetricResults generateMetricResults() throws Exception {
.setPayload(encodeStringSet(StringSetData.create(STRING_SET_VALUE)))
.build();
+ MetricsApi.MonitoringInfo boundedTrieMonitoringInfo =
+ MetricsApi.MonitoringInfo.newBuilder()
+ .setType(BOUNDED_TRIE_TYPE)
+ .putAllLabels(labelMap)
+ .setPayload(encodeBoundedTrie(new BoundedTrieData(ImmutableList.of("ab"))))
+ .build();
+
return JobApi.MetricResults.newBuilder()
.addAttempted(counterMonitoringInfo)
.addAttempted(distMonitoringInfo)
.addAttempted(gaugeMonitoringInfo)
.addAttempted(stringSetMonitoringInfo)
+ .addAttempted(boundedTrieMonitoringInfo)
.build();
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/BoundedTrie.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/BoundedTrie.java
new file mode 100644
index 000000000000..573c6fddab49
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/BoundedTrie.java
@@ -0,0 +1,54 @@
+/*
+ * 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 java.util.Arrays;
+import org.apache.beam.sdk.annotations.Internal;
+
+/**
+ * Internal: For internal use only and not for public consumption. This API is subject to
+ * incompatible changes, or even removal, in a future release.
+ *
+ * A metric that represents a bounded trie data structure. This interface extends the {@link
+ * Metric} interface and provides methods for adding string sequences (paths) to the trie.
+ *
+ *
The trie is bounded in size (max=100), meaning it has a maximum capacity for storing paths.
+ * When the trie reaches its capacity, it truncates paths. This is useful for tracking and
+ * aggregating a large number of distinct paths while limiting memory usage. It is not necessary but
+ * recommended that parts of paths provided as strings are hierarchical in nature so the truncation
+ * reduces granularity rather than complete data loss.
+ */
+@Internal
+public interface BoundedTrie extends Metric {
+
+ /**
+ * Adds a path to the trie. The path is represented as an iterable of string segments.
+ *
+ * @param values The segments of the path to add.
+ */
+ void add(Iterable values);
+
+ /**
+ * Adds a path to the trie. The path is represented as a variable number of string arguments.
+ *
+ * @param values The segments of the path to add.
+ */
+ default void add(String... values) {
+ add(Arrays.asList(values));
+ }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/BoundedTrieResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/BoundedTrieResult.java
new file mode 100644
index 000000000000..53d03f7a0b44
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/BoundedTrieResult.java
@@ -0,0 +1,57 @@
+/*
+ * 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.Set;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+
+/**
+ * Internal: For internal use only and not for public consumption. This API is subject to
+ * incompatible changes, or even removal, in a future release.
+ *
+ * The result of a {@link BoundedTrie} metric. The {@link BoundedTrieResult} hold an immutable
+ * copy of the set from which it was initially created representing that a result cannot be modified
+ * once created.
+ */
+@Internal
+@AutoValue
+public abstract class BoundedTrieResult {
+
+ public abstract Set> getResult();
+
+ /**
+ * Creates a {@link BoundedTrieResult} from the given {@link Set} by making an immutable copy.
+ *
+ * @param s the set from which the {@link BoundedTrieResult} should be created.
+ * @return {@link BoundedTrieResult} containing an immutable copy of the given set.
+ */
+ public static BoundedTrieResult create(Set> s) {
+ return new AutoValue_BoundedTrieResult(
+ ImmutableSet.copyOf(s.stream().map(ImmutableList::copyOf).collect(Collectors.toSet())));
+ }
+
+ /** @return an empty {@link BoundedTrieResult} */
+ public static BoundedTrieResult empty() {
+ return create(ImmutableSet.of());
+ }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricQueryResults.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricQueryResults.java
index 9f60ce3d6c07..476cfb58406a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricQueryResults.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricQueryResults.java
@@ -36,6 +36,9 @@ public abstract class MetricQueryResults {
/** Return the metric results for the sets that matched the filter. */
public abstract Iterable> getStringSets();
+ /** Return the metric results for the bounded tries that matched the filter. */
+ public abstract Iterable> getBoundedTries();
+
static void printMetrics(String type, Iterable> metrics, StringBuilder sb) {
List> metricsList = ImmutableList.copyOf(metrics);
if (!metricsList.isEmpty()) {
@@ -65,6 +68,7 @@ public final String toString() {
printMetrics("Distributions", getDistributions(), sb);
printMetrics("Gauges", getGauges(), sb);
printMetrics("StringSets", getStringSets(), sb);
+ printMetrics("BoundedTries", getBoundedTries(), sb);
sb.append(")");
return sb.toString();
}
@@ -73,7 +77,9 @@ public static MetricQueryResults create(
Iterable> counters,
Iterable> distributions,
Iterable> gauges,
- Iterable> stringSets) {
- return new AutoValue_MetricQueryResults(counters, distributions, gauges, stringSets);
+ Iterable> stringSets,
+ Iterable> boundedTries) {
+ return new AutoValue_MetricQueryResults(
+ counters, distributions, gauges, stringSets, boundedTries);
}
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java
index 6c8179006640..7c3b98e5bdac 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/Metrics.java
@@ -160,6 +160,22 @@ public static StringSet stringSet(Class> namespace, String name) {
return new DelegatingStringSet(MetricName.named(namespace, name));
}
+ /**
+ * Create a metric that accumulates and reports set of unique string values bounded to a max
+ * limit.
+ */
+ public static BoundedTrie boundedTrie(Class> namespace, String name) {
+ return new DelegatingBoundedTrie(MetricName.named(namespace, name));
+ }
+
+ /**
+ * Create a metric that accumulates and reports set of unique string values bounded to a max
+ * limit.
+ */
+ public static BoundedTrie boundedTrie(String namespace, String name) {
+ return new DelegatingBoundedTrie(MetricName.named(namespace, name));
+ }
+
/*
* A dedicated namespace for client throttling time. User DoFn can increment this metrics and then
* runner will put back pressure on scaling decision, if supported.
@@ -253,4 +269,28 @@ public MetricName getName() {
return name;
}
}
+
+ /**
+ * Implementation of {@link BoundedTrie} that delegates to the instance for the current context.
+ */
+ private static class DelegatingBoundedTrie implements Metric, BoundedTrie, Serializable {
+ private final MetricName name;
+
+ private DelegatingBoundedTrie(MetricName name) {
+ this.name = name;
+ }
+
+ @Override
+ public MetricName getName() {
+ return name;
+ }
+
+ @Override
+ public void add(Iterable values) {
+ MetricsContainer container = MetricsEnvironment.getCurrentContainer();
+ if (container != null) {
+ container.getBoundedTrie(name).add(values);
+ }
+ }
+ }
}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsContainer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsContainer.java
index 0c4766bb2c0b..2c3d9b3dc904 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsContainer.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricsContainer.java
@@ -59,6 +59,12 @@ default Counter getPerWorkerCounter(MetricName metricName) {
*/
StringSet getStringSet(MetricName metricName);
+ /**
+ * Return the {@link BoundedTrie} that should be used for implementing the given {@code
+ * metricName} in this container.
+ */
+ BoundedTrie getBoundedTrie(MetricName metricName);
+
/**
* Return the {@link Histogram} that should be used for implementing the given {@code metricName}
* in this container.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesBoundedTrieMetrics.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesBoundedTrieMetrics.java
new file mode 100644
index 000000000000..fe593531e1d3
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesBoundedTrieMetrics.java
@@ -0,0 +1,28 @@
+/*
+ * 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.testing;
+
+import org.apache.beam.sdk.annotations.Internal;
+
+/**
+ * Category tag for validation tests which utilize {@link org.apache.beam.sdk.metrics.BoundedTrie}.
+ * Tests tagged with {@link UsesBoundedTrieMetrics} should be run for runners which support
+ * BoundedTrie.
+ */
+@Internal
+public class UsesBoundedTrieMetrics {}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/BoundedTrieResultTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/BoundedTrieResultTest.java
new file mode 100644
index 000000000000..38887fdb0c0e
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/BoundedTrieResultTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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 java.util.List;
+import java.util.Set;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class BoundedTrieResultTest {
+
+ @Test
+ public void testCreate() {
+ Set> inputSet =
+ ImmutableSet.of(
+ ImmutableList.of("a", "b"), ImmutableList.of("c", "d"), ImmutableList.of("e"));
+ BoundedTrieResult result = BoundedTrieResult.create(inputSet);
+ Assert.assertEquals(inputSet, result.getResult());
+ }
+
+ @Test
+ public void testCreate_empty() {
+ Set> inputSet = ImmutableSet.of();
+ BoundedTrieResult result = BoundedTrieResult.create(inputSet);
+ Assert.assertEquals(inputSet, result.getResult());
+ }
+
+ @Test
+ public void testEmpty() {
+ BoundedTrieResult result = BoundedTrieResult.empty();
+ Assert.assertTrue(result.getResult().isEmpty());
+ }
+
+ @Test
+ public void testImmutability() {
+ Set> inputSet =
+ ImmutableSet.of(
+ ImmutableList.of("a", "b"), ImmutableList.of("c", "d"), ImmutableList.of("e"));
+ BoundedTrieResult result = BoundedTrieResult.create(inputSet);
+
+ // Try to modify the set returned by getResult()
+ try {
+ result.getResult().add(ImmutableList.of("f"));
+ Assert.fail("UnsupportedOperationException expected");
+ } catch (UnsupportedOperationException expected) {
+ }
+ }
+}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java
index 662c4f52628a..5d7527fa0990 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java
@@ -45,6 +45,7 @@
import org.apache.beam.sdk.testing.NeedsRunner;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.UsesAttemptedMetrics;
+import org.apache.beam.sdk.testing.UsesBoundedTrieMetrics;
import org.apache.beam.sdk.testing.UsesCommittedMetrics;
import org.apache.beam.sdk.testing.UsesCounterMetrics;
import org.apache.beam.sdk.testing.UsesDistributionMetrics;
@@ -122,6 +123,8 @@ public void startBundle() {
public void processElement(ProcessContext c) {
Distribution values = Metrics.distribution(MetricsTest.class, "input");
StringSet sources = Metrics.stringSet(MetricsTest.class, "sources");
+ BoundedTrie boundedTrieSources =
+ Metrics.boundedTrie(MetricsTest.class, "boundedTrieSources");
count.inc();
values.update(c.element());
@@ -131,6 +134,8 @@ public void processElement(ProcessContext c) {
sources.add("gcs"); // repeated should appear once
sources.add("gcs", "gcs"); // repeated should appear once
sideinputs.add("bigtable", "spanner");
+ boundedTrieSources.add(ImmutableList.of("ab_source", "cd_source"));
+ boundedTrieSources.add(ImmutableList.of("ef_source"));
}
@DoFn.FinishBundle
@@ -148,6 +153,8 @@ public void processElement(ProcessContext c) {
Distribution values = Metrics.distribution(MetricsTest.class, "input");
Gauge gauge = Metrics.gauge(MetricsTest.class, "my-gauge");
StringSet sinks = Metrics.stringSet(MetricsTest.class, "sinks");
+ BoundedTrie boundedTrieSinks =
+ Metrics.boundedTrie(MetricsTest.class, "boundedTrieSinks");
Integer element = c.element();
count.inc();
values.update(element);
@@ -155,6 +162,8 @@ public void processElement(ProcessContext c) {
c.output(element);
sinks.add("bq", "kafka", "kafka"); // repeated should appear once
sideinputs.add("bigtable", "sql");
+ boundedTrieSinks.add(ImmutableList.of("ab_sink", "cd_sink"));
+ boundedTrieSinks.add(ImmutableList.of("ef_sink"));
c.output(output2, element);
}
})
@@ -277,7 +286,8 @@ public static class CommittedMetricTests extends SharedTestBase {
UsesCounterMetrics.class,
UsesDistributionMetrics.class,
UsesGaugeMetrics.class,
- UsesStringSetMetrics.class
+ UsesStringSetMetrics.class,
+ UsesBoundedTrieMetrics.class
})
@Test
public void testAllCommittedMetrics() {
@@ -319,6 +329,14 @@ public void testCommittedStringSetMetrics() {
assertStringSetMetrics(metrics, true);
}
+ @Category({ValidatesRunner.class, UsesCommittedMetrics.class, UsesBoundedTrieMetrics.class})
+ @Test
+ public void testCommittedBoundedTrieMetrics() {
+ PipelineResult result = runPipelineWithMetrics();
+ MetricQueryResults metrics = queryTestMetrics(result);
+ assertBoundedTrieMetrics(metrics, true);
+ }
+
@Test
@Category({NeedsRunner.class, UsesAttemptedMetrics.class, UsesCounterMetrics.class})
public void testBoundedSourceMetrics() {
@@ -405,7 +423,8 @@ public static class AttemptedMetricTests extends SharedTestBase {
UsesCounterMetrics.class,
UsesDistributionMetrics.class,
UsesGaugeMetrics.class,
- UsesStringSetMetrics.class
+ UsesStringSetMetrics.class,
+ UsesBoundedTrieMetrics.class
})
@Test
public void testAllAttemptedMetrics() {
@@ -448,6 +467,14 @@ public void testAttemptedStringSetMetrics() {
assertStringSetMetrics(metrics, false);
}
+ @Category({ValidatesRunner.class, UsesAttemptedMetrics.class, UsesBoundedTrieMetrics.class})
+ @Test
+ public void testAttemptedBoundedTrieMetrics() {
+ PipelineResult result = runPipelineWithMetrics();
+ MetricQueryResults metrics = queryTestMetrics(result);
+ assertBoundedTrieMetrics(metrics, false);
+ }
+
@Test
@Category({ValidatesRunner.class, UsesAttemptedMetrics.class, UsesCounterMetrics.class})
public void testBoundedSourceMetricsInSplit() {
@@ -634,6 +661,34 @@ private static void assertStringSetMetrics(MetricQueryResults metrics, boolean i
isCommitted)));
}
+ private static void assertBoundedTrieMetrics(MetricQueryResults metrics, boolean isCommitted) {
+ // TODO(https://github.com/apache/beam/issues/32001) use containsInAnyOrder once portableMetrics
+ // duplicate metrics issue fixed
+ assertThat(
+ metrics.getBoundedTries(),
+ hasItem(
+ metricsResultPatchStep(
+ "boundedTrieSources",
+ "MyStep1",
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ ImmutableList.of("ab_source", "cd_source", String.valueOf(false)),
+ ImmutableList.of("ef_source", String.valueOf(false)))),
+ isCommitted)));
+ assertThat(
+ metrics.getBoundedTries(),
+ hasItem(
+ metricsResult(
+ NAMESPACE,
+ "boundedTrieSinks",
+ "MyStep2",
+ BoundedTrieResult.create(
+ ImmutableSet.of(
+ ImmutableList.of("ab_sink", "cd_sink", String.valueOf(false)),
+ ImmutableList.of("ef_sink", String.valueOf(false)))),
+ isCommitted)));
+ }
+
private static void assertDistributionMetrics(MetricQueryResults metrics, boolean isCommitted) {
assertThat(
metrics.getDistributions(),
@@ -665,5 +720,6 @@ private static void assertAllMetrics(MetricQueryResults metrics, boolean isCommi
assertDistributionMetrics(metrics, isCommitted);
assertGaugeMetrics(metrics, isCommitted);
assertStringSetMetrics(metrics, isCommitted);
+ assertBoundedTrieMetrics(metrics, isCommitted);
}
}
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java
index bcd243ba746d..11b9bc49448c 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java
@@ -36,6 +36,7 @@
import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo;
import org.apache.beam.runners.core.metrics.MetricsContainerStepMap;
import org.apache.beam.runners.core.metrics.MonitoringInfoEncodings;
+import org.apache.beam.sdk.metrics.BoundedTrie;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.Distribution;
import org.apache.beam.sdk.metrics.Gauge;
@@ -225,6 +226,14 @@ public StringSet getStringSet(MetricName metricName) {
return tracker.metricsContainerRegistry.getUnboundContainer().getStringSet(metricName);
}
+ @Override
+ public BoundedTrie getBoundedTrie(MetricName metricName) {
+ if (tracker.currentState != null) {
+ return tracker.currentState.metricsContainer.getBoundedTrie(metricName);
+ }
+ return tracker.metricsContainerRegistry.getUnboundContainer().getBoundedTrie(metricName);
+ }
+
@Override
public Histogram getHistogram(MetricName metricName, HistogramData.BucketType bucketType) {
if (tracker.currentState != null) {
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java
index 1f4341860295..76eda6b92772 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java
@@ -34,6 +34,8 @@
import org.apache.beam.fn.harness.control.ExecutionStateSampler.ExecutionStateTracker;
import org.apache.beam.fn.harness.control.ExecutionStateSampler.ExecutionStateTrackerStatus;
import org.apache.beam.runners.core.metrics.MonitoringInfoEncodings;
+import org.apache.beam.sdk.metrics.BoundedTrie;
+import org.apache.beam.sdk.metrics.BoundedTrieResult;
import org.apache.beam.sdk.metrics.Counter;
import org.apache.beam.sdk.metrics.DelegatingHistogram;
import org.apache.beam.sdk.metrics.Distribution;
@@ -47,6 +49,7 @@
import org.apache.beam.sdk.testing.ExpectedLogs;
import org.apache.beam.sdk.util.HistogramData;
import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
import org.joda.time.DateTimeUtils.MillisProvider;
import org.joda.time.Duration;
@@ -69,6 +72,8 @@ public class ExecutionStateSamplerTest {
private static final Gauge TEST_USER_GAUGE = Metrics.gauge("foo", "gauge");
private static final StringSet TEST_USER_STRING_SET = Metrics.stringSet("foo", "stringset");
+ private static final BoundedTrie TEST_USER_BOUNDED_TRIE =
+ Metrics.boundedTrie("foo", "boundedtrie");
private static final Histogram TEST_USER_HISTOGRAM =
new DelegatingHistogram(
MetricName.named("foo", "histogram"), HistogramData.LinearBuckets.of(0, 100, 1), false);
@@ -380,6 +385,7 @@ public void testCountersReturnedAreBasedUponCurrentExecutionState() throws Excep
TEST_USER_DISTRIBUTION.update(2);
TEST_USER_GAUGE.set(3);
TEST_USER_STRING_SET.add("ab");
+ TEST_USER_BOUNDED_TRIE.add("bt_ab");
TEST_USER_HISTOGRAM.update(4);
state.deactivate();
@@ -387,6 +393,7 @@ public void testCountersReturnedAreBasedUponCurrentExecutionState() throws Excep
TEST_USER_DISTRIBUTION.update(12);
TEST_USER_GAUGE.set(13);
TEST_USER_STRING_SET.add("cd");
+ TEST_USER_BOUNDED_TRIE.add("bt_cd");
TEST_USER_HISTOGRAM.update(14);
TEST_USER_HISTOGRAM.update(14);
@@ -425,6 +432,14 @@ public void testCountersReturnedAreBasedUponCurrentExecutionState() throws Excep
.getStringSet(TEST_USER_STRING_SET.getName())
.getCumulative()
.stringSet());
+ assertEquals(
+ BoundedTrieResult.create(ImmutableSet.of(ImmutableList.of("bt_ab", String.valueOf(false)))),
+ tracker
+ .getMetricsContainerRegistry()
+ .getContainer("ptransformId")
+ .getBoundedTrie(TEST_USER_BOUNDED_TRIE.getName())
+ .getCumulative()
+ .extractResult());
assertEquals(
1L,
(long)
@@ -471,6 +486,14 @@ public void testCountersReturnedAreBasedUponCurrentExecutionState() throws Excep
.getStringSet(TEST_USER_STRING_SET.getName())
.getCumulative()
.stringSet());
+ assertEquals(
+ BoundedTrieResult.create(ImmutableSet.of(ImmutableList.of("bt_cd", String.valueOf(false)))),
+ tracker
+ .getMetricsContainerRegistry()
+ .getUnboundContainer()
+ .getBoundedTrie(TEST_USER_BOUNDED_TRIE.getName())
+ .getCumulative()
+ .extractResult());
assertEquals(
2L,
(long)