29
29
import java .util .HashMap ;
30
30
import java .util .HashSet ;
31
31
import java .util .List ;
32
+ import java .util .Map ;
32
33
import org .apache .beam .model .pipeline .v1 .MetricsApi .BoundedTrie ;
33
34
import org .apache .beam .model .pipeline .v1 .RunnerApi ;
34
35
import org .apache .beam .runners .core .metrics .BoundedTrieData ;
44
45
import org .apache .beam .sdk .metrics .MetricsFilter ;
45
46
import org .apache .beam .sdk .metrics .StringSetResult ;
46
47
import org .apache .beam .sdk .runners .AppliedPTransform ;
48
+ import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .annotations .VisibleForTesting ;
47
49
import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .base .Objects ;
48
50
import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .collect .BiMap ;
49
51
import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .collect .ImmutableList ;
52
+ import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .collect .ImmutableMap ;
50
53
import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .collect .ImmutableSet ;
51
54
import org .checkerframework .checker .nullness .qual .Nullable ;
52
55
import org .slf4j .Logger ;
@@ -133,7 +136,8 @@ private JobMetrics getJobMetrics() throws IOException {
133
136
return result ;
134
137
}
135
138
136
- private static class DataflowMetricResultExtractor {
139
+ @ VisibleForTesting
140
+ static class DataflowMetricResultExtractor {
137
141
private final ImmutableList .Builder <MetricResult <Long >> counterResults ;
138
142
private final ImmutableList .Builder <MetricResult <DistributionResult >> distributionResults ;
139
143
private final ImmutableList .Builder <MetricResult <GaugeResult >> gaugeResults ;
@@ -206,12 +210,21 @@ private StringSetResult getStringSetValue(MetricUpdate metricUpdate) {
206
210
}
207
211
208
212
private BoundedTrieResult getBoundedTrieValue (MetricUpdate metricUpdate ) {
209
- if (metricUpdate .getTrie () == null ) {
213
+ BoundedTrieData trieData = null ;
214
+ Object trieFromResponse = metricUpdate .getTrie ();
215
+ // Fail-safely cast Trie returned by dataflow API to BoundedTrieResult
216
+ if (trieFromResponse instanceof BoundedTrie ) {
217
+ BoundedTrie bTrie = (BoundedTrie ) metricUpdate .getTrie ();
218
+ trieData = BoundedTrieData .fromProto (bTrie );
219
+ } else if (trieFromResponse instanceof com .google .protobuf .Struct ) {
220
+ trieData = trieFromStruct ((com .google .protobuf .Struct ) trieFromResponse );
221
+ }
222
+
223
+ if (trieData != null ) {
224
+ return BoundedTrieResult .create (trieData .extractResult ().getResult ());
225
+ } else {
210
226
return BoundedTrieResult .empty ();
211
227
}
212
- BoundedTrie bTrie = (BoundedTrie ) metricUpdate .getTrie ();
213
- BoundedTrieData trieData = BoundedTrieData .fromProto (bTrie );
214
- return BoundedTrieResult .create (trieData .extractResult ().getResult ());
215
228
}
216
229
217
230
private DistributionResult getDistributionValue (MetricUpdate metricUpdate ) {
@@ -226,6 +239,68 @@ private DistributionResult getDistributionValue(MetricUpdate metricUpdate) {
226
239
return DistributionResult .create (sum , count , min , max );
227
240
}
228
241
242
+ /** Translate Struct proto returned by Dataflow API client to BoundedTrieData. */
243
+ @ VisibleForTesting
244
+ @ SuppressWarnings ("ReferenceEquality" ) // compare with protobuf Struct default instance
245
+ static BoundedTrieData trieFromStruct (com .google .protobuf .Struct responseProto ) {
246
+ Map <String , com .google .protobuf .Value > fieldsMap = responseProto .getFieldsMap ();
247
+ int bound = 0 ;
248
+ List <String > singleton = null ;
249
+ com .google .protobuf .Value maybeBound = fieldsMap .get ("bound" );
250
+ if (maybeBound != null ) {
251
+ bound = (int ) maybeBound .getNumberValue ();
252
+ }
253
+ com .google .protobuf .Value maybeSingleton = fieldsMap .get ("singleton" );
254
+ if (maybeSingleton != null ) {
255
+ List <com .google .protobuf .Value > valueList = maybeSingleton .getListValue ().getValuesList ();
256
+ ImmutableList .Builder <String > builder = ImmutableList .builder ();
257
+ for (com .google .protobuf .Value stringValue : valueList ) {
258
+ builder .add (stringValue .getStringValue ());
259
+ }
260
+ singleton = builder .build ();
261
+ }
262
+ com .google .protobuf .Value maybeRoot = fieldsMap .get ("root" );
263
+ BoundedTrieData .BoundedTrieNode root = null ;
264
+ if (maybeRoot != null
265
+ && maybeRoot .getStructValue () != com .google .protobuf .Struct .getDefaultInstance ()) {
266
+ root = trieNodeFromStruct (maybeRoot .getStructValue ());
267
+ }
268
+ return new BoundedTrieData (singleton , root , bound );
269
+ }
270
+
271
+ /**
272
+ * Translate Struct proto returned by Dataflow API client to BoundedTrieData.BoundedTrieNode.
273
+ */
274
+ @ SuppressWarnings ("ReferenceEquality" ) // compare with protobuf Struct default instance
275
+ private static BoundedTrieData .BoundedTrieNode trieNodeFromStruct (
276
+ com .google .protobuf .Struct responseProto ) {
277
+ Map <String , com .google .protobuf .Value > fieldsMap = responseProto .getFieldsMap ();
278
+ boolean truncated = false ;
279
+ com .google .protobuf .Value mayTruncated = fieldsMap .get ("truncated" );
280
+ if (mayTruncated != null ) {
281
+ truncated = mayTruncated .getBoolValue ();
282
+ }
283
+ int childrenSize = 0 ;
284
+ ImmutableMap .Builder <String , BoundedTrieData .BoundedTrieNode > builder =
285
+ ImmutableMap .builder ();
286
+ com .google .protobuf .Value maybeChildren = fieldsMap .get ("children" );
287
+ if (maybeChildren != null ) {
288
+ Map <String , com .google .protobuf .Value > allChildren =
289
+ maybeChildren .getStructValue ().getFieldsMap ();
290
+ for (Map .Entry <String , com .google .protobuf .Value > childValue : allChildren .entrySet ()) {
291
+ com .google .protobuf .Struct maybeChild = childValue .getValue ().getStructValue ();
292
+ if (maybeChild != com .google .protobuf .Struct .getDefaultInstance ()) {
293
+ BoundedTrieData .BoundedTrieNode child =
294
+ trieNodeFromStruct (childValue .getValue ().getStructValue ());
295
+ builder .put (childValue .getKey (), child );
296
+ childrenSize += child .getSize ();
297
+ }
298
+ }
299
+ }
300
+ Map <String , BoundedTrieData .BoundedTrieNode > children = builder .build ();
301
+ return new BoundedTrieData .BoundedTrieNode (children , truncated , Math .max (1 , childrenSize ));
302
+ }
303
+
229
304
public Iterable <MetricResult <DistributionResult >> getDistributionResults () {
230
305
return distributionResults .build ();
231
306
}
0 commit comments