github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/metrics/execution.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 # cython: language_level=3 19 20 """ 21 This module is for internal use only; no backwards-compatibility guarantees. 22 23 The classes in this file keep shared state, and organize metrics information. 24 25 Available classes: 26 27 - MetricKey - Internal key for a metric. 28 - MetricResult - Current status of a metric's updates/commits. 29 - _MetricsEnvironment - Keeps track of MetricsContainer and other metrics 30 information for every single execution working thread. 31 - MetricsContainer - Holds the metrics of a single step and a single 32 unit-of-commit (bundle). 33 """ 34 35 # pytype: skip-file 36 37 import threading 38 from typing import TYPE_CHECKING 39 from typing import Any 40 from typing import Dict 41 from typing import FrozenSet 42 from typing import Optional 43 from typing import Type 44 from typing import Union 45 from typing import cast 46 47 from apache_beam.metrics import monitoring_infos 48 from apache_beam.metrics.cells import CounterCell 49 from apache_beam.metrics.cells import DistributionCell 50 from apache_beam.metrics.cells import GaugeCell 51 from apache_beam.runners.worker import statesampler 52 from apache_beam.runners.worker.statesampler import get_current_tracker 53 54 if TYPE_CHECKING: 55 from apache_beam.metrics.cells import GaugeData 56 from apache_beam.metrics.cells import DistributionData 57 from apache_beam.metrics.cells import MetricCell 58 from apache_beam.metrics.cells import MetricCellFactory 59 from apache_beam.metrics.metricbase import MetricName 60 from apache_beam.portability.api import metrics_pb2 61 62 63 class MetricKey(object): 64 """Key used to identify instance of metric cell. 65 66 Metrics are internally keyed by the name of the step they're associated with, 67 the name and namespace (if it is a user defined metric) of the metric, 68 and any extra label metadata added by the runner specific metric collection 69 service. 70 """ 71 def __init__(self, step, metric, labels=None): 72 """Initializes ``MetricKey``. 73 74 Args: 75 step: A string with the step this metric cell is part of. 76 metric: A ``MetricName`` namespace+name that identifies a metric. 77 labels: An arbitrary set of labels that also identifies the metric. 78 """ 79 self.step = step 80 self.metric = metric 81 self.labels = labels if labels else {} 82 83 def __eq__(self, other): 84 return ( 85 self.step == other.step and self.metric == other.metric and 86 self.labels == other.labels) 87 88 def __hash__(self): 89 return hash((self.step, self.metric, frozenset(self.labels))) 90 91 def __repr__(self): 92 return 'MetricKey(step={}, metric={}, labels={})'.format( 93 self.step, self.metric, self.labels) 94 95 96 class MetricResult(object): 97 """Keeps track of the status of a metric within a single bundle. 98 99 It contains the physical and logical updates to the metric. Physical updates 100 are updates that have not necessarily been committed, but that have been made 101 during pipeline execution. Logical updates are updates that have been 102 committed. 103 104 Attributes: 105 key: A ``MetricKey`` that identifies the metric and bundle of this result. 106 committed: The committed updates of the metric. This attribute's type is 107 of metric type result (e.g. int, DistributionResult, GaugeResult). 108 attempted: The logical updates of the metric. This attribute's type is that 109 of metric type result (e.g. int, DistributionResult, GaugeResult). 110 """ 111 def __init__(self, key, committed, attempted): 112 """Initializes ``MetricResult``. 113 Args: 114 key: A ``MetricKey`` object. 115 committed: Metric data that has been committed (e.g. logical updates) 116 attempted: Metric data that has been attempted (e.g. physical updates) 117 """ 118 self.key = key 119 self.committed = committed 120 self.attempted = attempted 121 122 def __eq__(self, other): 123 return ( 124 self.key == other.key and self.committed == other.committed and 125 self.attempted == other.attempted) 126 127 def __hash__(self): 128 return hash((self.key, self.committed, self.attempted)) 129 130 def __repr__(self): 131 return 'MetricResult(key={}, committed={}, attempted={})'.format( 132 self.key, str(self.committed), str(self.attempted)) 133 134 def __str__(self): 135 return repr(self) 136 137 @property 138 def result(self): 139 """Short-hand for falling back to attempted metrics if it seems that 140 committed was not populated (e.g. due to not being supported on a given 141 runner""" 142 return self.committed if self.committed else self.attempted 143 144 145 class _MetricsEnvironment(object): 146 """Holds the MetricsContainer for every thread and other metric information. 147 148 This class is not meant to be instantiated, instead being used to keep 149 track of global state. 150 """ 151 def current_container(self): 152 """Returns the current MetricsContainer.""" 153 sampler = statesampler.get_current_tracker() 154 if sampler is None: 155 return None 156 return sampler.current_state().metrics_container 157 158 def process_wide_container(self): 159 """Returns the MetricsContainer for process wide metrics, e.g. memory.""" 160 return PROCESS_WIDE_METRICS_CONTAINER 161 162 163 MetricsEnvironment = _MetricsEnvironment() 164 165 166 class _TypedMetricName(object): 167 """Like MetricName, but also stores the cell type of the metric.""" 168 def __init__( 169 self, 170 cell_type, # type: Union[Type[MetricCell], MetricCellFactory] 171 metric_name # type: Union[str, MetricName] 172 ): 173 # type: (...) -> None 174 self.cell_type = cell_type 175 self.metric_name = metric_name 176 if isinstance(metric_name, str): 177 self.fast_name = metric_name 178 else: 179 self.fast_name = metric_name.fast_name() 180 # Cached for speed, as this is used as a key for every counter update. 181 self._hash = hash((cell_type, self.fast_name)) 182 183 def __eq__(self, other): 184 return self is other or ( 185 self.cell_type == other.cell_type and self.fast_name == other.fast_name) 186 187 def __hash__(self): 188 return self._hash 189 190 def __str__(self): 191 return '%s %s' % (self.cell_type, self.metric_name) 192 193 def __reduce__(self): 194 return _TypedMetricName, (self.cell_type, self.metric_name) 195 196 197 _DEFAULT = None # type: Any 198 199 200 class MetricUpdater(object): 201 """A callable that updates the metric as quickly as possible.""" 202 def __init__( 203 self, 204 cell_type, # type: Union[Type[MetricCell], MetricCellFactory] 205 metric_name, # type: Union[str, MetricName] 206 default_value=None, 207 process_wide=False): 208 self.process_wide = process_wide 209 self.typed_metric_name = _TypedMetricName(cell_type, metric_name) 210 self.default_value = default_value 211 212 def __call__(self, value=_DEFAULT): 213 # type: (Any) -> None 214 if value is _DEFAULT: 215 if self.default_value is _DEFAULT: 216 raise ValueError( 217 'Missing value for update of %s' % self.typed_metric_name.fast_name) 218 value = self.default_value 219 if self.process_wide: 220 MetricsEnvironment.process_wide_container().get_metric_cell( 221 self.typed_metric_name).update(value) 222 else: 223 tracker = get_current_tracker() 224 if tracker is not None: 225 tracker.update_metric(self.typed_metric_name, value) 226 227 def __reduce__(self): 228 return MetricUpdater, ( 229 self.typed_metric_name.cell_type, 230 self.typed_metric_name.metric_name, 231 self.default_value) 232 233 234 class MetricsContainer(object): 235 """Holds the metrics of a single step and a single bundle. 236 237 Or the metrics associated with the process/SDK harness. I.e. memory usage. 238 """ 239 def __init__(self, step_name): 240 self.step_name = step_name 241 self.lock = threading.Lock() 242 self.metrics = {} # type: Dict[_TypedMetricName, MetricCell] 243 244 def get_counter(self, metric_name): 245 # type: (MetricName) -> CounterCell 246 return cast( 247 CounterCell, 248 self.get_metric_cell(_TypedMetricName(CounterCell, metric_name))) 249 250 def get_distribution(self, metric_name): 251 # type: (MetricName) -> DistributionCell 252 return cast( 253 DistributionCell, 254 self.get_metric_cell(_TypedMetricName(DistributionCell, metric_name))) 255 256 def get_gauge(self, metric_name): 257 # type: (MetricName) -> GaugeCell 258 return cast( 259 GaugeCell, 260 self.get_metric_cell(_TypedMetricName(GaugeCell, metric_name))) 261 262 def get_metric_cell(self, typed_metric_name): 263 # type: (_TypedMetricName) -> MetricCell 264 cell = self.metrics.get(typed_metric_name, None) 265 if cell is None: 266 with self.lock: 267 cell = self.metrics[typed_metric_name] = typed_metric_name.cell_type() 268 return cell 269 270 def get_cumulative(self): 271 # type: () -> MetricUpdates 272 273 """Return MetricUpdates with cumulative values of all metrics in container. 274 275 This returns all the cumulative values for all metrics. 276 """ 277 counters = { 278 MetricKey(self.step_name, k.metric_name): v.get_cumulative() 279 for k, 280 v in self.metrics.items() if k.cell_type == CounterCell 281 } 282 283 distributions = { 284 MetricKey(self.step_name, k.metric_name): v.get_cumulative() 285 for k, 286 v in self.metrics.items() if k.cell_type == DistributionCell 287 } 288 289 gauges = { 290 MetricKey(self.step_name, k.metric_name): v.get_cumulative() 291 for k, 292 v in self.metrics.items() if k.cell_type == GaugeCell 293 } 294 295 return MetricUpdates(counters, distributions, gauges) 296 297 def to_runner_api(self): 298 return [ 299 cell.to_runner_api_user_metric(key.metric_name) for key, 300 cell in self.metrics.items() 301 ] 302 303 def to_runner_api_monitoring_infos(self, transform_id): 304 # type: (str) -> Dict[FrozenSet, metrics_pb2.MonitoringInfo] 305 306 """Returns a list of MonitoringInfos for the metrics in this container.""" 307 with self.lock: 308 items = list(self.metrics.items()) 309 all_metrics = [ 310 cell.to_runner_api_monitoring_info(key.metric_name, transform_id) 311 for key, 312 cell in items 313 ] 314 return { 315 monitoring_infos.to_key(mi): mi 316 for mi in all_metrics if mi is not None 317 } 318 319 def reset(self): 320 # type: () -> None 321 322 """Reset all metrics in the MetricsContainer. This does not delete added 323 metrics. 324 """ 325 326 for metric in self.metrics.values(): 327 metric.reset() 328 329 def __reduce__(self): 330 raise NotImplementedError 331 332 333 PROCESS_WIDE_METRICS_CONTAINER = MetricsContainer(None) 334 335 336 class MetricUpdates(object): 337 """Contains updates for several metrics. 338 339 A metric update is an object containing information to update a metric. 340 For Distribution metrics, it is DistributionData, and for Counter metrics, 341 it's an int. 342 """ 343 def __init__( 344 self, 345 counters=None, # type: Optional[Dict[MetricKey, int]] 346 distributions=None, # type: Optional[Dict[MetricKey, DistributionData]] 347 gauges=None # type: Optional[Dict[MetricKey, GaugeData]] 348 ): 349 # type: (...) -> None 350 351 """Create a MetricUpdates object. 352 353 Args: 354 counters: Dictionary of MetricKey:MetricUpdate updates. 355 distributions: Dictionary of MetricKey:MetricUpdate objects. 356 gauges: Dictionary of MetricKey:MetricUpdate objects. 357 """ 358 self.counters = counters or {} 359 self.distributions = distributions or {} 360 self.gauges = gauges or {}