y1chi commented on a change in pull request #14482:
URL: https://github.com/apache/beam/pull/14482#discussion_r611847137



##########
File path: 
runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java
##########
@@ -56,4 +58,12 @@ public void reset() {
     }
     return monitoringInfos;
   }
+
+  public Map<String, ByteString> getExecutionTimeMonitoringData(ShortIdMap 
shortIds) {
+    ImmutableMap.Builder<String, ByteString> builder = ImmutableMap.builder();
+    for (SimpleExecutionState state : executionStates) {
+      builder.put(state.getTotalMillisShortId(shortIds), 
state.getTotalMillisPayload());

Review comment:
       Should we filter out the the state with zero total mills to improve 
efficiency? It can be common in streaming and I think they don't make a 
difference.

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
##########
@@ -369,29 +371,46 @@ private void 
createRunnerAndConsumersForPTransformRecursively(
           
.setProcessBundleProgress(BeamFnApi.ProcessBundleProgressResponse.getDefaultInstance());
     }
 
-    // TODO(BEAM-6597): This should really only be reporting monitoring infos 
where the data
-    // changed
-    // and we should be using the short id system.
+    Map<String, ByteString> monitoringData = monitoringData(bundleProcessor);
+    if (runnerAcceptsShortIds) {
+      response.putAllMonitoringData(monitoringData);
+    } else {
+      for (Map.Entry<String, ByteString> metric : 
response.getMonitoringDataMap().entrySet()) {

Review comment:
       for (Map.Entry<String, ByteString> metric : monitoringData.entrySet()) ?

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
##########
@@ -318,23 +328,15 @@ private void 
createRunnerAndConsumersForPTransformRecursively(
         // Add all checkpointed residuals to the response.
         
response.addAllResidualRoots(bundleProcessor.getSplitListener().getResidualRoots());
 
-        // TODO(BEAM-6597): This should be reporting monitoring infos using 
the short id system.
-        // Get start bundle Execution Time Metrics.
-        response.addAllMonitoringInfos(
-            
bundleProcessor.getStartFunctionRegistry().getExecutionTimeMonitoringInfos());
-        // Get process bundle Execution Time Metrics.
-        response.addAllMonitoringInfos(
-            
bundleProcessor.getpCollectionConsumerRegistry().getExecutionTimeMonitoringInfos());
-        // Get finish bundle Execution Time Metrics.
-        response.addAllMonitoringInfos(
-            
bundleProcessor.getFinishFunctionRegistry().getExecutionTimeMonitoringInfos());
-        // Extract MonitoringInfos that come from the metrics container 
registry.
-        response.addAllMonitoringInfos(
-            
bundleProcessor.getMetricsContainerRegistry().getMonitoringInfos());
-        // Add any additional monitoring infos that the "runners" report 
explicitly.
-        for (ProgressRequestCallback progressRequestCallback :
-            bundleProcessor.getProgressRequestCallbacks()) {
-          
response.addAllMonitoringInfos(progressRequestCallback.getMonitoringInfos());
+        // Add all metrics to the response.
+        Map<String, ByteString> monitoringData = 
monitoringData(bundleProcessor);
+        if (runnerAcceptsShortIds) {
+          response.putAllMonitoringData(monitoringData);
+        } else {
+          for (Map.Entry<String, ByteString> metric : 
response.getMonitoringDataMap().entrySet()) {

Review comment:
       for (Map.Entry<String, ByteString> metric : monitoringData.entrySet()) ?

##########
File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
##########
@@ -369,29 +371,46 @@ private void 
createRunnerAndConsumersForPTransformRecursively(
           
.setProcessBundleProgress(BeamFnApi.ProcessBundleProgressResponse.getDefaultInstance());
     }
 
-    // TODO(BEAM-6597): This should really only be reporting monitoring infos 
where the data
-    // changed
-    // and we should be using the short id system.
+    Map<String, ByteString> monitoringData = monitoringData(bundleProcessor);
+    if (runnerAcceptsShortIds) {
+      response.putAllMonitoringData(monitoringData);
+    } else {
+      for (Map.Entry<String, ByteString> metric : 
response.getMonitoringDataMap().entrySet()) {
+        response.addMonitoringInfos(
+            
shortIds.get(metric.getKey()).toBuilder().setPayload(metric.getValue()));
+      }
+    }
 
+    return 
BeamFnApi.InstructionResponse.newBuilder().setProcessBundleProgress(response);
+  }
+
+  private ImmutableMap<String, ByteString> monitoringData(BundleProcessor 
bundleProcessor)
+      throws Exception {
+    ImmutableMap.Builder<String, ByteString> result = ImmutableMap.builder();
     // Get start bundle Execution Time Metrics.
-    response.addAllMonitoringInfos(
-        
bundleProcessor.getStartFunctionRegistry().getExecutionTimeMonitoringInfos());
+    result.putAll(
+        
bundleProcessor.getStartFunctionRegistry().getExecutionTimeMonitoringData(shortIds));
     // Get process bundle Execution Time Metrics.
-    response.addAllMonitoringInfos(
-        
bundleProcessor.getpCollectionConsumerRegistry().getExecutionTimeMonitoringInfos());
+    result.putAll(
+        
bundleProcessor.getpCollectionConsumerRegistry().getExecutionTimeMonitoringData(shortIds));
     // Get finish bundle Execution Time Metrics.
-    response.addAllMonitoringInfos(
-        
bundleProcessor.getFinishFunctionRegistry().getExecutionTimeMonitoringInfos());
-    // Extract all other MonitoringInfos other than the execution time 
monitoring infos.
-    response.addAllMonitoringInfos(
-        bundleProcessor.getMetricsContainerRegistry().getMonitoringInfos());
+    result.putAll(
+        
bundleProcessor.getFinishFunctionRegistry().getExecutionTimeMonitoringData(shortIds));
+    // Extract MonitoringInfos that come from the metrics container registry.
+    
result.putAll(bundleProcessor.getMetricsContainerRegistry().getMonitoringData(shortIds));
     // Add any additional monitoring infos that the "runners" report 
explicitly.
     for (ProgressRequestCallback progressRequestCallback :
         bundleProcessor.getProgressRequestCallbacks()) {
-      
response.addAllMonitoringInfos(progressRequestCallback.getMonitoringInfos());
+      // TODO(BEAM-6597): This should be reporting monitoring infos using the 
short id system.

Review comment:
       Isn't it already using the short id?




-- 
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.

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


Reply via email to