Skip to content

Parse struct returned from Dataflow API to BoundedTrieData #34738

New issue

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

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

Already on GitHub? Sign in to your account

Draft
wants to merge 2 commits into
base: master
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -311,7 +311,7 @@ public final String toString() {
* intended to be used directly outside of {@link BoundedTrieData} with multiple threads.
*/
@VisibleForTesting
static class BoundedTrieNode implements Serializable {
public static class BoundedTrieNode implements Serializable {

public static final String TRUNCATED_TRUE = String.valueOf(true);
public static final String TRUNCATED_FALSE = String.valueOf(false);
Expand All @@ -334,7 +334,7 @@ static class BoundedTrieNode implements Serializable {
private int size;

/** Constructs an empty `BoundedTrieNode` with size 1 and not truncated. */
BoundedTrieNode() {
public BoundedTrieNode() {
this(new HashMap<>(), false, 1);
}

Expand All @@ -345,7 +345,8 @@ static class BoundedTrieNode implements Serializable {
* @param truncated Whether this node is truncated.
* @param size The size of the subtree rooted at this node.
*/
BoundedTrieNode(@Nonnull Map<String, BoundedTrieNode> children, boolean truncated, int size) {
public BoundedTrieNode(
@Nonnull Map<String, BoundedTrieNode> children, boolean truncated, int size) {
this.children = children;
this.size = size;
this.truncated = truncated;
Expand Down Expand Up @@ -561,7 +562,7 @@ boolean contains(List<String> segments) {
*
* @return The size of the subtree.
*/
int getSize() {
public int getSize() {
return size;
}

Expand Down
1 change: 1 addition & 0 deletions runners/google-cloud-dataflow-java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,7 @@ dependencies {
implementation library.java.google_auth_library_oauth2_http
implementation library.java.google_http_client
implementation library.java.google_http_client_gson
implementation library.java.protobuf_java
permitUnusedDeclared library.java.google_http_client_gson // BEAM-11761
implementation library.java.hamcrest
implementation library.java.jackson_annotations
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import org.apache.beam.model.pipeline.v1.MetricsApi.BoundedTrie;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.runners.core.metrics.BoundedTrieData;
Expand All @@ -44,9 +45,11 @@
import org.apache.beam.sdk.metrics.MetricsFilter;
import org.apache.beam.sdk.metrics.StringSetResult;
import org.apache.beam.sdk.runners.AppliedPTransform;
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.base.Objects;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap;
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.checkerframework.checker.nullness.qual.Nullable;
import org.slf4j.Logger;
Expand Down Expand Up @@ -133,7 +136,8 @@ private JobMetrics getJobMetrics() throws IOException {
return result;
}

private static class DataflowMetricResultExtractor {
@VisibleForTesting
static class DataflowMetricResultExtractor {
private final ImmutableList.Builder<MetricResult<Long>> counterResults;
private final ImmutableList.Builder<MetricResult<DistributionResult>> distributionResults;
private final ImmutableList.Builder<MetricResult<GaugeResult>> gaugeResults;
Expand Down Expand Up @@ -206,12 +210,21 @@ private StringSetResult getStringSetValue(MetricUpdate metricUpdate) {
}

private BoundedTrieResult getBoundedTrieValue(MetricUpdate metricUpdate) {
if (metricUpdate.getTrie() == null) {
BoundedTrieData trieData = null;
Object trieFromResponse = metricUpdate.getTrie();
// Fail-safely cast Trie returned by dataflow API to BoundedTrieResult
if (trieFromResponse instanceof BoundedTrie) {
BoundedTrie bTrie = (BoundedTrie) metricUpdate.getTrie();
trieData = BoundedTrieData.fromProto(bTrie);
} else if (trieFromResponse instanceof com.google.protobuf.Struct) {
trieData = trieFromStruct((com.google.protobuf.Struct) trieFromResponse);
}

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

private DistributionResult getDistributionValue(MetricUpdate metricUpdate) {
Expand All @@ -226,6 +239,68 @@ private DistributionResult getDistributionValue(MetricUpdate metricUpdate) {
return DistributionResult.create(sum, count, min, max);
}

/** Translate Struct proto returned by Dataflow API client to BoundedTrieData. */
@VisibleForTesting
@SuppressWarnings("ReferenceEquality") // Compare with protobuf Struct default instance
static BoundedTrieData trieFromStruct(com.google.protobuf.Struct responseProto) {
Map<String, com.google.protobuf.Value> fieldsMap = responseProto.getFieldsMap();
int bound = 0;
List<String> singleton = null;
com.google.protobuf.Value maybeBound = fieldsMap.get("bound");
if (maybeBound != null) {
bound = (int) maybeBound.getNumberValue();
}
com.google.protobuf.Value maybeSingleton = fieldsMap.get("singleton");
if (maybeSingleton != null) {
List<com.google.protobuf.Value> valueList = maybeSingleton.getListValue().getValuesList();
ImmutableList.Builder<String> builder = ImmutableList.builder();
for (com.google.protobuf.Value stringValue : valueList) {
builder.add(stringValue.getStringValue());
}
singleton = builder.build();
}
com.google.protobuf.Value maybeRoot = fieldsMap.get("root");
BoundedTrieData.BoundedTrieNode root = null;
if (maybeRoot != null
&& maybeRoot.getStructValue() != com.google.protobuf.Struct.getDefaultInstance()) {
root = trieNodeFromStruct(maybeRoot.getStructValue());
}
return new BoundedTrieData(singleton, root, bound);
}

/**
* Translate Struct proto returned by Dataflow API client to BoundedTrieData.BoundedTrieNode.
*/
@SuppressWarnings({"ReferenceEquality"}) // Compare with protobuf Struct default instance
private static BoundedTrieData.BoundedTrieNode trieNodeFromStruct(
com.google.protobuf.Struct responseProto) {
Map<String, com.google.protobuf.Value> fieldsMap = responseProto.getFieldsMap();
boolean truncated = false;
com.google.protobuf.Value mayTruncated = fieldsMap.get("truncated");
if (mayTruncated != null) {
truncated = mayTruncated.getBoolValue();
}
int childrenSize = 0;
ImmutableMap.Builder<String, BoundedTrieData.BoundedTrieNode> builder =
ImmutableMap.builder();
com.google.protobuf.Value maybeChildren = fieldsMap.get("children");
if (maybeChildren != null) {
Map<String, com.google.protobuf.Value> allChildren =
maybeChildren.getStructValue().getFieldsMap();
for (Map.Entry<String, com.google.protobuf.Value> childValue : allChildren.entrySet()) {
com.google.protobuf.Struct maybeChild = childValue.getValue().getStructValue();
if (maybeChild != com.google.protobuf.Struct.getDefaultInstance()) {
BoundedTrieData.BoundedTrieNode child =
trieNodeFromStruct(childValue.getValue().getStructValue());
builder.put(childValue.getKey(), child);
childrenSize += child.getSize();
}
}
}
Map<String, BoundedTrieData.BoundedTrieNode> children = builder.build();
return new BoundedTrieData.BoundedTrieNode(children, truncated, Math.max(1, childrenSize));
}

public Iterable<MetricResult<DistributionResult>> getDistributionResults() {
return distributionResults.build();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
Expand All @@ -38,6 +39,8 @@
import com.google.api.services.dataflow.model.JobMetrics;
import com.google.api.services.dataflow.model.MetricStructuredName;
import com.google.api.services.dataflow.model.MetricUpdate;
import com.google.protobuf.TextFormat;
import com.google.protobuf.TextFormat.ParseException;
import java.io.IOException;
import java.math.BigDecimal;
import java.util.Set;
Expand Down Expand Up @@ -296,6 +299,129 @@ public void testSingleStringSetUpdates() throws IOException {
StringSetResult.create(ImmutableSet.of("ab", "cd")))));
}

@Test
public void testSingletonBoundedTrieFromMessage() throws ParseException {
String textProto =
" fields {\n"
+ " key: \"bound\"\n"
+ " value {\n"
+ " number_value: 100\n"
+ " }\n"
+ " }\n"
+ " fields {\n"
+ " key: \"singleton\"\n"
+ " value {\n"
+ " list_value {\n"
+ " values {\n"
+ " string_value: \"pubsub:\"\n"
+ " }\n"
+ " values {\n"
+ " string_value: \"topic:\"\n"
+ " }\n"
+ " values {\n"
+ " string_value: \"`google.com:abc`.\"\n"
+ " }\n"
+ " values {\n"
+ " string_value: \"some-topic\"\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }";
com.google.protobuf.Struct response =
TextFormat.parse(textProto, com.google.protobuf.Struct.class);
BoundedTrieData result = DataflowMetrics.DataflowMetricResultExtractor.trieFromStruct(response);
assertEquals(
"BoundedTrieData({'pubsub:topic:`google.com:abc`.some-topicfalse'})", result.toString());
}

@Test
public void testNestedBoundedTrieFromMessage() throws ParseException {
String textProto =
"fields {\n"
+ " key: \"bound\"\n"
+ " value {\n"
+ " number_value: 100\n"
+ " }\n"
+ "}\n"
+ "fields {\n"
+ " key: \"root\"\n"
+ " value {\n"
+ " struct_value {\n"
+ " fields {\n"
+ " key: \"children\"\n"
+ " value {\n"
+ " struct_value {\n"
+ " fields {\n"
+ " key: \"gcs:\"\n"
+ " value {\n"
+ " struct_value {\n"
+ " fields {\n"
+ " key: \"children\"\n"
+ " value {\n"
+ " struct_value {\n"
+ " fields {\n"
+ " key: \"some-bucket.\"\n"
+ " value {\n"
+ " struct_value {\n"
+ " fields {\n"
+ " key: \"children\"\n"
+ " value {\n"
+ " struct_value {\n"
+ " fields {\n"
+ " key: \"some-folder/\"\n"
+ " value {\n"
+ " struct_value {\n"
+ " fields {\n"
+ " key: \"truncated\"\n"
+ " value {\n"
+ " bool_value: true\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " fields {\n"
+ " key: \"truncated\"\n"
+ " value {\n"
+ " bool_value: false\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " fields {\n"
+ " key: \"truncated\"\n"
+ " value {\n"
+ " bool_value: false\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " fields {\n"
+ " key: \"truncated\"\n"
+ " value {\n"
+ " bool_value: false\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ " }\n"
+ "}";
com.google.protobuf.Struct response =
TextFormat.parse(textProto, com.google.protobuf.Struct.class);
BoundedTrieData result = DataflowMetrics.DataflowMetricResultExtractor.trieFromStruct(response);
assertEquals("BoundedTrieData({'gcs:some-bucket.some-folder/true'})", result.toString());
}

@Test
public void testSingleBoundedTrieUpdates() throws IOException {
AppliedPTransform<?, ?, ?> myStep = mock(AppliedPTransform.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,8 @@
<suppress id="ForbidNonVendoredGrpcProtobuf" files=".*testinfra.*mockapis.*" />
<suppress id="ForbidNonVendoredGrpcProtobuf" files=".*requestresponse.*" />
<suppress id="ForbidNonVendoredGrpcProtobuf" files=".*examples.*webapis.*" />
<suppress id="ForbidNonVendoredGrpcProtobuf" files=".*runners.*dataflow.*DataflowMetrics\.java" />
<suppress id="ForbidNonVendoredGrpcProtobuf" files=".*runners.*dataflow.*DataflowMetricsTest\.java" />

<!-- Flink -->
<!-- Checkstyle does not correctly detect package files across multiple source directories. -->
Expand Down
Loading