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

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

                Author: ASF GitHub Bot
            Created on: 24/Oct/19 16:35
            Start Date: 24/Oct/19 16:35
    Worklog Time Spent: 10m 
      Work Description: ajamato commented on pull request #9843: [BEAM-4775] 
Converting MonitoringInfos to MetricResults in PortableRunner
URL: https://github.com/apache/beam/pull/9843#discussion_r338676240
 
 

 ##########
 File path: sdks/python/apache_beam/runners/portability/portable_metrics.py
 ##########
 @@ -0,0 +1,74 @@
+#
+# 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.
+#
+
+from __future__ import absolute_import
+
+from apache_beam.metrics import monitoring_infos
+from apache_beam.metrics.execution import MetricKey
+from apache_beam.metrics.metric import MetricName
+
+
+class ParseMonitoringInfoMixin(object):
+  @staticmethod
+  def from_monitoring_infos(monitoring_info_list, user_metrics_only=False):
+    """Groups MonitoringInfo objects into counters, distributions and gauges.
+
+    Args:
+      monitoring_info_list: An iterable of MonitoringInfo objects.
+      user_metrics_only: If true, includes user metrics only.
+    """
+    counters = {}
+    distributions = {}
+    gauges = {}
+
+    for mi in monitoring_info_list:
+      if (user_metrics_only and
+          not monitoring_infos.is_user_monitoring_info(mi)):
+        continue
+
+      key = ParseMonitoringInfoMixin._create_metric_key(mi)
+      metric_result = (monitoring_infos.extract_metric_result_map_value(mi))
+
+      if monitoring_infos.is_counter(mi):
+        counters[key] = metric_result
+      elif monitoring_infos.is_distribution(mi):
+        distributions[key] = metric_result
+      elif monitoring_infos.is_gauge(mi):
+        gauges[key] = metric_result
+
+    return counters, distributions, gauges
+
+  @staticmethod
+  def _create_metric_key(monitoring_info):
+    step_name = ParseMonitoringInfoMixin._get_step_name(monitoring_info)
+    namespace, name = 
monitoring_infos.parse_namespace_and_name(monitoring_info)
+    return MetricKey(step_name, MetricName(namespace, name))
+
+  @staticmethod
+  def _get_step_name(monitoring_info):
+    keys_to_check = [monitoring_infos.PTRANSFORM_LABEL,
 
 Review comment:
   No, the intention is that URN + Labels defines the metric instance.
   
   Think of URN as the class of metric, and URN+Labels defines the object 
instance, as an analogy.
   
   I don't quite remember exactly what MetricKey contained, but generally our 
collection objects for metrics need to account for URN+Labels to correctly 
identify the metric instance.
   
   MetricResult was originally designed just for user metrics, which did not 
have labels. Just a name and namespace. The labels concept was introduced to 
MonitoringInfos later. Then the name and namespace were reworked to be labels.
   See the user metric MonitoringInfoSpec, defining what should be populated on 
an MI
   
https://github.com/apache/beam/blob/d4afbabf38a3ab557625c9c091ed5f06ca6731ce/model/pipeline/src/main/proto/metrics.proto#L67
 
----------------------------------------------------------------
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]


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

    Worklog Id:     (was: 333573)
    Time Spent: 52h 20m  (was: 52h 10m)

> JobService should support returning metrics
> -------------------------------------------
>
>                 Key: BEAM-4775
>                 URL: https://issues.apache.org/jira/browse/BEAM-4775
>             Project: Beam
>          Issue Type: Bug
>          Components: beam-model
>            Reporter: Eugene Kirpichov
>            Assignee: Kamil Wasilewski
>            Priority: Major
>          Time Spent: 52h 20m
>  Remaining Estimate: 0h
>
> Design doc: [https://s.apache.org/get-metrics-api].
> Further discussion is ongoing on [this 
> doc|https://docs.google.com/document/d/1m83TsFvJbOlcLfXVXprQm1B7vUakhbLZMzuRrOHWnTg/edit?ts=5c826bb4#heading=h.faqan9rjc6dm].
> We want to report job metrics back to the portability harness from the runner 
> harness, for displaying to users.
> h1. Relevant PRs in flight:
> h2. Ready for Review:
>  * [#8022|https://github.com/apache/beam/pull/8022]: correct the Job RPC 
> protos from [#8018|https://github.com/apache/beam/pull/8018].
> h2. Iterating / Discussing:
>  * [#7971|https://github.com/apache/beam/pull/7971]: Flink portable metrics: 
> get ptransform from MonitoringInfo, not stage name
>  ** this is a simpler, Flink-specific PR that is basically duplicated inside 
> each of the following two, so may be worth trying to merge in first
>  * #[7915|https://github.com/apache/beam/pull/7915]: use MonitoringInfo data 
> model in Java SDK metrics
>  * [#7868|https://github.com/apache/beam/pull/7868]: MonitoringInfo URN tweaks
> h2. Merged
>  * [#8018|https://github.com/apache/beam/pull/8018]: add job metrics RPC 
> protos
>  * [#7867|https://github.com/apache/beam/pull/7867]: key MetricResult by a 
> MetricKey
>  * [#7938|https://github.com/apache/beam/pull/7938]: move MonitoringInfo 
> protos to model/pipeline module
>  * [#7883|https://github.com/apache/beam/pull/7883]: Add 
> MetricQueryResults.allMetrics() helper
>  * [#7866|https://github.com/apache/beam/pull/7866]: move function helpers 
> from fn-harness to sdks/java/core
>  * [#7890|https://github.com/apache/beam/pull/7890]: consolidate MetricResult 
> implementations
> h2. Closed
>  * [#7934|https://github.com/apache/beam/pull/7934]: job metrics RPC + SDK 
> support
>  * [#7876|https://github.com/apache/beam/pull/7876]: Clean up metric protos; 
> support integer distributions, gauges



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to