github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/runners/portability/portable_metrics.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 # pytype: skip-file 19 20 import logging 21 22 from apache_beam.metrics import monitoring_infos 23 from apache_beam.metrics.execution import MetricKey 24 from apache_beam.metrics.metric import MetricName 25 26 _LOGGER = logging.getLogger(__name__) 27 28 29 def from_monitoring_infos(monitoring_info_list, user_metrics_only=False): 30 """Groups MonitoringInfo objects into counters, distributions and gauges. 31 32 Args: 33 monitoring_info_list: An iterable of MonitoringInfo objects. 34 user_metrics_only: If true, includes user metrics only. 35 Returns: 36 A tuple containing three dictionaries: counters, distributions and gauges, 37 respectively. Each dictionary contains (MetricKey, metric result) pairs. 38 """ 39 counters = {} 40 distributions = {} 41 gauges = {} 42 43 for mi in monitoring_info_list: 44 if (user_metrics_only and not monitoring_infos.is_user_monitoring_info(mi)): 45 continue 46 47 try: 48 key = _create_metric_key(mi) 49 except ValueError as e: 50 _LOGGER.debug(str(e)) 51 continue 52 metric_result = (monitoring_infos.extract_metric_result_map_value(mi)) 53 54 if monitoring_infos.is_counter(mi): 55 counters[key] = metric_result 56 elif monitoring_infos.is_distribution(mi): 57 distributions[key] = metric_result 58 elif monitoring_infos.is_gauge(mi): 59 gauges[key] = metric_result 60 61 return counters, distributions, gauges 62 63 64 def _create_metric_key(monitoring_info): 65 step_name = monitoring_infos.get_step_name(monitoring_info) 66 namespace, name = monitoring_infos.parse_namespace_and_name(monitoring_info) 67 return MetricKey(step_name, MetricName(namespace, name))