github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/internal/metrics/cells.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 """ 19 This file contains internal metric cell classes. A metric cell is used to 20 accumulate in-memory changes to a metric. It represents a specific metric 21 in a single context. 22 23 For internal use only. No backwards compatibility guarantees. 24 """ 25 26 # pytype: skip-file 27 28 from typing import TYPE_CHECKING 29 from typing import Optional 30 31 from apache_beam.metrics.cells import MetricAggregator 32 from apache_beam.metrics.cells import MetricCell 33 from apache_beam.metrics.cells import MetricCellFactory 34 from apache_beam.utils.histogram import Histogram 35 36 if TYPE_CHECKING: 37 from apache_beam.utils.histogram import BucketType 38 39 40 class HistogramCell(MetricCell): 41 """For internal use only; no backwards-compatibility guarantees. 42 43 Tracks the current value and delta for a histogram metric. 44 45 Each cell tracks the state of a metric independently per context per bundle. 46 Therefore, each metric has a different cell in each bundle, that is later 47 aggregated. 48 49 This class is thread safe since underlying histogram object is thread safe. 50 """ 51 def __init__(self, bucket_type): 52 self._bucket_type = bucket_type 53 self.data = HistogramAggregator(bucket_type).identity_element() 54 55 def reset(self): 56 self.data = HistogramAggregator(self._bucket_type).identity_element() 57 58 def combine(self, other): 59 # type: (HistogramCell) -> HistogramCell 60 result = HistogramCell(self._bucket_type) 61 result.data = self.data.combine(other.data) 62 return result 63 64 def update(self, value): 65 self.data.histogram.record(value) 66 67 def get_cumulative(self): 68 # type: () -> HistogramData 69 return self.data.get_cumulative() 70 71 def to_runner_api_monitoring_info(self, name, transform_id): 72 # Histogram metric is currently worker-local and internal 73 # use only. This method should be implemented when runners 74 # support Histogram metric reporting. 75 return None 76 77 78 class HistogramCellFactory(MetricCellFactory): 79 def __init__(self, bucket_type): 80 self._bucket_type = bucket_type 81 82 def __call__(self): 83 return HistogramCell(self._bucket_type) 84 85 def __eq__(self, other): 86 if not isinstance(other, HistogramCellFactory): 87 return False 88 return self._bucket_type == other._bucket_type 89 90 def __hash__(self): 91 return hash(self._bucket_type) 92 93 94 class HistogramResult(object): 95 def __init__(self, data): 96 # type: (HistogramData) -> None 97 self.data = data 98 99 def __eq__(self, other): 100 if isinstance(other, HistogramResult): 101 return self.data == other.data 102 else: 103 return False 104 105 def __hash__(self): 106 return hash(self.data) 107 108 def __repr__(self): 109 return '<HistogramResult({})>'.format( 110 self.data.histogram.get_percentile_info()) 111 112 @property 113 def p99(self): 114 return self.data.histogram.p99() 115 116 @property 117 def p95(self): 118 return self.data.histogram.p95() 119 120 @property 121 def p90(self): 122 return self.data.histogram.p90() 123 124 125 class HistogramData(object): 126 """For internal use only; no backwards-compatibility guarantees. 127 128 The data structure that holds data about a histogram metric. 129 130 This object is not thread safe, so it's not supposed to be modified 131 outside the HistogramCell. 132 """ 133 def __init__(self, histogram): 134 self.histogram = histogram 135 136 def __eq__(self, other): 137 return self.histogram == other.histogram 138 139 def __hash__(self): 140 return hash(self.histogram) 141 142 def __repr__(self): 143 return 'HistogramData({})'.format(self.histogram.get_percentile_info()) 144 145 def get_cumulative(self): 146 # type: () -> HistogramData 147 return HistogramData(self.histogram) 148 149 def combine(self, other): 150 # type: (Optional[HistogramData]) -> HistogramData 151 if other is None: 152 return self 153 154 return HistogramData(self.histogram.combine(other.histogram)) 155 156 157 class HistogramAggregator(MetricAggregator): 158 """For internal use only; no backwards-compatibility guarantees. 159 160 Aggregator for Histogram metric data during pipeline execution. 161 162 Values aggregated should be ``HistogramData`` objects. 163 """ 164 def __init__(self, bucket_type): 165 # type: (BucketType) -> None 166 self._bucket_type = bucket_type 167 168 def identity_element(self): 169 # type: () -> HistogramData 170 return HistogramData(Histogram(self._bucket_type)) 171 172 def combine(self, x, y): 173 # type: (HistogramData, HistogramData) -> HistogramData 174 return x.combine(y) 175 176 def result(self, x): 177 # type: (HistogramData) -> HistogramResult 178 return HistogramResult(x.get_cumulative())