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)