rohitsinha54 commented on code in PR #33385:
URL: https://github.com/apache/beam/pull/33385#discussion_r1887304439


##########
runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/BoundedTrieData.java:
##########
@@ -0,0 +1,593 @@
+/*
+ * 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 com.google.auto.value.AutoValue;
+import com.google.common.base.Preconditions;
+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.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.ImmutableMap;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+
+/**
+ * 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.
+ *
+ * <p>The trie can be in one of two states:
+ *
+ * <ul>
+ *   <li>**Singleton:** Contains a single path.
+ *   <li>**Trie:** Contains a {@link BoundedTrieNode} representing the root of 
the trie.
+ * </ul>
+ */
+@AutoValue
+public abstract class BoundedTrieData implements Serializable {
+
+  private static final int DEFAULT_BOUND = 100; // Default maximum size of the 
trie
+
+  /**
+   * Returns an {@link Optional} containing the singleton path if this {@link 
BoundedTrieData}
+   * represents a single path.
+   */
+  public abstract Optional<List<String>> singleton();
+
+  /**
+   * Returns an {@link Optional} containing the root {@link BoundedTrieNode} 
if this {@link
+   * BoundedTrieData} represents a trie.
+   */
+  public abstract Optional<BoundedTrieNode> root();
+
+  /** Returns the maximum size of the trie. */
+  public abstract int bound();
+
+  /**
+   * Creates a {@link BoundedTrieData} instance.
+   *
+   * @param singleton The singleton path (optional).
+   * @param root The root node of the trie (optional).
+   * @param bound The maximum size of the trie.
+   * @throws IllegalArgumentException If both or neither of {@code singleton} 
and {@code root} are
+   *     specified or both are null, or if {@code bound} is less than 1.
+   */
+  public static BoundedTrieData create(
+      @Nullable List<String> singleton, @Nullable BoundedTrieNode root, int 
bound) {
+    Preconditions.checkArgument(
+        (singleton == null ^ root == null),
+        "Either and only one of singleton or root must be specified.");
+    Preconditions.checkArgument(bound >= 1, "Bound must be at least 1.");
+    return new AutoValue_BoundedTrieData(
+        Optional.ofNullable(singleton), Optional.ofNullable(root), bound);
+  }
+
+  /**
+   * Creates a {@link BoundedTrieData} instance from a {@link BoundedTrieNode} 
with the default
+   * bound.
+   *
+   * @param root The root node of the trie.
+   */
+  public static BoundedTrieData create(@Nonnull BoundedTrieNode root) {
+    return create(null, root, DEFAULT_BOUND);
+  }
+
+  /**
+   * Creates a {@link BoundedTrieData} instance from a singleton path with the 
default bound.
+   *
+   * @param singleton The singleton path.
+   */
+  public static BoundedTrieData create(@Nonnull List<String> singleton) {
+    return create(singleton, null, DEFAULT_BOUND);
+  }
+
+  /** Converts this {@link BoundedTrieData} to its proto {@link BoundedTrie}. 
*/
+  public BoundedTrie toProto() {
+    BoundedTrie.Builder builder = BoundedTrie.newBuilder();
+    builder.setBound(bound());
+    singleton().ifPresent(builder::addAllSingleton);
+    root().ifPresent(r -> builder.setRoot(r.toProto()));
+    return builder.build();
+  }
+
+  /** Creates a {@link BoundedTrieData} instance from its proto {@link 
BoundedTrie}. */
+  public static BoundedTrieData fromProto(BoundedTrie proto) {
+    List<String> singleton = proto.getSingletonList();
+    BoundedTrieNode root = proto.hasRoot() ? 
BoundedTrieNode.fromProto(proto.getRoot()) : null;
+    return create(singleton, root, proto.getBound());
+  }
+
+  /** Returns this {@link BoundedTrieData} as a {@link BoundedTrieNode}. */
+  public BoundedTrieNode asTrie() {
+    return root()
+        .orElseGet(
+            () -> {
+              BoundedTrieNode newRoot = new BoundedTrieNode();
+              singleton().ifPresent(newRoot::add);
+              return newRoot;
+            });
+  }
+
+  /** Returns a new {@link BoundedTrieData} instance that is a deep copy of 
this instance. */
+  public BoundedTrieData getCumulative() {
+    return root().isPresent()
+        ? create(null, new BoundedTrieNode(root().get()), bound())
+        : create(singleton().get(), null, bound());
+  }
+
+  /** Extracts the data from this {@link BoundedTrieData} as a {@link 
BoundedTrieResult}. */
+  public BoundedTrieResult getBoundedTrieResult() {

Review Comment:
   This kinda seem redundant, BoundedTrieResult just a wrapper on the result. I 
don't remember from StringSet changes if XYZResult classes are useful in e2e 
metric plumbing or not. When I do that and this seem like a wrapper only I will 
get rid of it so devs don't have to do .getBoundedTrieResult.getResult. Unless 
there is objection around differing from standard pattern 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to