[ 
https://issues.apache.org/jira/browse/BEAM-6181?focusedWorklogId=177732&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-177732
 ]

ASF GitHub Bot logged work on BEAM-6181:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 20/Dec/18 19:45
            Start Date: 20/Dec/18 19:45
    Worklog Time Spent: 10m 
      Work Description: ajamato commented on a change in pull request #7323: 
[BEAM-6181] Implemented msec counters support in FnApi world.
URL: https://github.com/apache/beam/pull/7323#discussion_r243374184
 
 

 ##########
 File path: 
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemStatusClient.java
 ##########
 @@ -266,27 +266,62 @@ private synchronized WorkItemStatus 
createStatusUpdate(boolean isFinal) {
     return status;
   }
 
-  // todo(migryz) this method should return List<CounterUpdate> instead of 
updating member variable
   @VisibleForTesting
   synchronized void populateCounterUpdates(WorkItemStatus status) {
     if (worker == null) {
       return;
     }
 
+    // Checking against boolean, because getCompleted can return null?
     boolean isFinalUpdate = Boolean.TRUE.equals(status.getCompleted());
 
-    ImmutableList.Builder<CounterUpdate> counterUpdatesListBuilder = 
ImmutableList.builder();
+    Map<Object, CounterUpdate> counterUpdatesMap = new HashMap<>();
     // Output counters
-    
counterUpdatesListBuilder.addAll(extractCounters(worker.getOutputCounters()));
+    extractCounters(worker.getOutputCounters())
+        .forEach(
+            x ->
+                counterUpdatesMap.put(
+                    x.getStructuredNameAndMetadata() == null
+                        ? x.getNameAndKind()
+                        : x.getStructuredNameAndMetadata(),
+                    x));
+
     // User metrics reported in Worker
-    counterUpdatesListBuilder.addAll(extractMetrics(isFinalUpdate));
+    extractMetrics(isFinalUpdate)
+        .forEach(
+            x ->
+                counterUpdatesMap.put(
+                    x.getStructuredNameAndMetadata() == null
+                        ? x.getNameAndKind()
+                        : x.getStructuredNameAndMetadata(),
+                    x));
+
+    // counterUpdatesListBuilder.addAll(extractMetrics(isFinalUpdate));
     // MSec counters reported in worker
-    counterUpdatesListBuilder.addAll(extractMsecCounters(isFinalUpdate));
-    // Metrics reported in SDK runner.
-    counterUpdatesListBuilder.addAll(worker.extractMetricUpdates());
+    extractMsecCounters(isFinalUpdate)
+        .forEach(
+            x ->
+                counterUpdatesMap.put(
+                    x.getStructuredNameAndMetadata() == null
+                        ? x.getNameAndKind()
+                        : x.getStructuredNameAndMetadata(),
+                    x));
 
-    ImmutableList<CounterUpdate> counterUpdates = 
counterUpdatesListBuilder.build();
 
 Review comment:
   Can you log error if we try to add a duplicate to this map. This is one of 
the failure cases you were afraid of. It would be better if we could add a 
flag/experiment or something around the new functionality and only enable it in 
the portable runner. I don't want this to accidentally break the existing code.
   
   Is there a way to know this code is running portability mode? If you know 
that then you can accept metrics conditionaly from diff sources

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 177732)
    Time Spent: 9h 50m  (was: 9h 40m)

> Utilize MetricInfo for reporting user metrics in Portable Dataflow Java 
> Runner.
> -------------------------------------------------------------------------------
>
>                 Key: BEAM-6181
>                 URL: https://issues.apache.org/jira/browse/BEAM-6181
>             Project: Beam
>          Issue Type: Bug
>          Components: java-fn-execution
>            Reporter: Mikhail Gryzykhin
>            Assignee: Mikhail Gryzykhin
>            Priority: Major
>          Time Spent: 9h 50m
>  Remaining Estimate: 0h
>
> New approach to report metrics in FnApi is to utilize MetricInfo structures.
> This approach is implemented in Python SDK and work is ongoing in Java SDK.
> This tasks includes plumbing User metrics reported via MetricInfos through 
> Dataflow Java Runner. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to