github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/transforms/py_dataflow_distribution_counter.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 """For internal use only; no backwards-compatibility guarantees.""" 19 20 # pytype: skip-file 21 22 globals()['INT64_MAX'] = 2**63 - 1 23 globals()['INT64_MIN'] = -2**63 24 25 POWER_TEN = [ 26 10e-1, 27 10e0, 28 10e1, 29 10e2, 30 10e3, 31 10e4, 32 10e5, 33 10e6, 34 10e7, 35 10e8, 36 10e9, 37 10e10, 38 10e11, 39 10e12, 40 10e13, 41 10e14, 42 10e15, 43 10e16, 44 10e17, 45 10e18 46 ] 47 48 49 def get_log10_round_to_floor(element): 50 power = 0 51 while element >= POWER_TEN[power]: 52 power += 1 53 return power - 1 54 55 56 class DataflowDistributionCounter(object): 57 """Pure python DataflowDistributionCounter in case Cython not available. 58 59 60 Please avoid using python mode if possible, since it's super slow 61 Cythonized DatadflowDistributionCounter defined in 62 apache_beam.transforms.cy_dataflow_distribution_counter. 63 64 Currently using special bucketing strategy suitable for Dataflow 65 66 Attributes: 67 min: minimum value of all inputs. 68 max: maximum value of all inputs. 69 count: total count of all inputs. 70 sum: sum of all inputs. 71 buckets: histogram buckets of value counts for a 72 distribution(1,2,5 bucketing). Max bucket_index is 58( sys.maxint as input). 73 is_cythonized: mark whether DataflowDistributionCounter cythonized. 74 """ 75 # Assume the max input is sys.maxint, then the possible max bucket size is 59 76 MAX_BUCKET_SIZE = 59 77 78 # 3 buckets for every power of ten -> 1, 2, 5 79 BUCKET_PER_TEN = 3 80 81 def __init__(self): 82 global INT64_MAX # pylint: disable=global-variable-not-assigned 83 self.min = INT64_MAX 84 self.max = 0 85 self.count = 0 86 self.sum = 0 87 self.buckets = [0] * self.MAX_BUCKET_SIZE 88 self.is_cythonized = False 89 90 def add_input(self, element): 91 if element < 0: 92 raise ValueError('Distribution counters support only non-negative value') 93 self.min = min(self.min, element) 94 self.max = max(self.max, element) 95 self.count += 1 96 self.sum += element 97 bucket_index = self.calculate_bucket_index(element) 98 self.buckets[bucket_index] += 1 99 100 def add_input_n(self, element, n): 101 if element < 0: 102 raise ValueError('Distribution counters support only non-negative value') 103 self.min = min(self.min, element) 104 self.max = max(self.max, element) 105 self.count += n 106 self.sum += element * n 107 bucket_index = self.calculate_bucket_index(element) 108 self.buckets[bucket_index] += n 109 110 def calculate_bucket_index(self, element): 111 """Calculate the bucket index for the given element.""" 112 if element == 0: 113 return 0 114 log10_floor = get_log10_round_to_floor(element) 115 power_of_ten = POWER_TEN[log10_floor] 116 if element < power_of_ten * 2: 117 bucket_offset = 0 118 elif element < power_of_ten * 5: 119 bucket_offset = 1 120 else: 121 bucket_offset = 2 122 return 1 + log10_floor * self.BUCKET_PER_TEN + bucket_offset 123 124 def translate_to_histogram(self, histogram): 125 """Translate buckets into Histogram. 126 127 Args: 128 histogram: apache_beam.runners.dataflow.internal.clents.dataflow.Histogram 129 Ideally, only call this function when reporting counter to 130 dataflow service. 131 """ 132 first_bucket_offset = 0 133 last_bucket_offset = 0 134 for index in range(0, self.MAX_BUCKET_SIZE): 135 if self.buckets[index] != 0: 136 first_bucket_offset = index 137 break 138 for index in range(self.MAX_BUCKET_SIZE - 1, -1, -1): 139 if self.buckets[index] != 0: 140 last_bucket_offset = index 141 break 142 histogram.firstBucketOffset = first_bucket_offset 143 histogram.bucketCounts = ( 144 self.buckets[first_bucket_offset:last_bucket_offset + 1]) 145 146 def extract_output(self): 147 global INT64_MIN # pylint: disable=global-variable-not-assigned 148 global INT64_MAX # pylint: disable=global-variable-not-assigned 149 if not INT64_MIN <= self.sum <= INT64_MAX: 150 self.sum %= 2**64 151 if self.sum >= INT64_MAX: 152 self.sum -= 2**64 153 mean = self.sum // self.count if self.count else float('nan') 154 return mean, self.sum, self.count, self.min, self.max 155 156 def merge(self, accumulators): 157 raise NotImplementedError()