github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/transforms/cy_dataflow_distribution_counter.pyx (about) 1 # Licensed to the Apache Software Foundation (ASF) under one or more 2 # contributor license agreements. See the NOTICE file distributed with 3 # this work for additional information regarding copyright ownership. 4 # The ASF licenses this file to You under the Apache License, Version 2.0 5 # (the "License"); you may not use this file except in compliance with 6 # the License. You may obtain a copy of the License at 7 # 8 # http://www.apache.org/licenses/LICENSE-2.0 9 # 10 # Unless required by applicable law or agreed to in writing, software 11 # distributed under the License is distributed on an "AS IS" BASIS, 12 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 # See the License for the specific language governing permissions and 14 # limitations under the License. 15 # 16 17 # cython: profile=True 18 # cython: language_level=3 19 20 """ For internal use only. No backwards compatibility guarantees.""" 21 22 cimport cython 23 from libc.stdint cimport int64_t, INT64_MAX 24 from libc.stdlib cimport calloc, free 25 26 27 cdef unsigned long long* POWER_TEN = [10e-1, 10e0, 10e1, 10e2, 10e3, 10e4, 10e5, 28 10e6, 10e7, 10e8, 10e9, 10e10, 10e11, 29 10e12, 10e13, 10e14, 10e15, 10e16, 10e17, 30 10e18] 31 32 33 cdef int64_t get_log10_round_to_floor(int64_t element): 34 cdef int power = 0 35 while element >= POWER_TEN[power]: 36 power += 1 37 return power - 1 38 39 40 cdef class DataflowDistributionCounter(object): 41 """Distribution Counter: 42 43 Contains value distribution statistics and methods for incrementing. 44 45 Currently using special bucketing strategy suitable for Dataflow 46 47 Attributes: 48 min: minimum value of all inputs. 49 max: maximum value of all inputs. 50 count: total count of all inputs. 51 sum: sum of all inputs. 52 buckets: histogram buckets of value counts for a 53 distribution(1,2,5 bucketing). Max bucket_index is 58( sys.maxint as input). 54 is_cythonized: mark whether DataflowDistributionCounter cythonized. 55 """ 56 def __init__(self): 57 self.min = INT64_MAX 58 self.max = 0 59 self.count = 0 60 self.sum = 0 61 self.buckets = <int64_t*> calloc(MAX_BUCKET_SIZE, sizeof(int64_t)) 62 self.is_cythonized = True 63 64 def __dealloc__(self): 65 """free allocated memory""" 66 free(self.buckets) 67 68 cpdef bint add_input(self, int64_t element) except -1: 69 if element < 0: 70 raise ValueError('Distribution counters support only non-negative value') 71 self.min = min(self.min, element) 72 self.max = max(self.max, element) 73 self.count += 1 74 self.sum += element 75 cdef int64_t bucket_index = self._fast_calculate_bucket_index(element) 76 self.buckets[bucket_index] += 1 77 78 cpdef bint add_input_n(self, int64_t element, int64_t n) except -1: 79 if element < 0: 80 raise ValueError('Distribution counters support only non-negative value') 81 self.min = min(self.min, element) 82 self.max = max(self.max, element) 83 self.count += n 84 self.sum += element*n 85 cdef int64_t bucket_index = self._fast_calculate_bucket_index(element) 86 self.buckets[bucket_index] += n 87 88 cdef int64_t _fast_calculate_bucket_index(self, int64_t element): 89 """Calculate the bucket index for the given element. 90 91 Declare calculate_bucket_index as cdef in order to improve performance, 92 since cpdef will have significant overhead. 93 """ 94 if element == 0: 95 return 0 96 cdef int64_t log10_floor = get_log10_round_to_floor(element) 97 cdef int64_t power_of_ten = POWER_TEN[log10_floor] 98 cdef int64_t bucket_offset = 0 99 if element < power_of_ten * 2: 100 bucket_offset = 0 101 elif element < power_of_ten * 5: 102 bucket_offset = 1 103 else: 104 bucket_offset = 2 105 return 1 + log10_floor * BUCKET_PER_TEN + bucket_offset 106 107 cpdef void translate_to_histogram(self, histogram): 108 """Translate buckets into Histogram. 109 110 Args: 111 histogram: apache_beam.runners.dataflow.internal.clents.dataflow.Histogram 112 Ideally, only call this function when reporting counter to 113 dataflow service. 114 """ 115 cdef int first_bucket_offset = 0 116 cdef int last_bucket_offset = 0 117 cdef int index = 0 118 for index in range(0, MAX_BUCKET_SIZE): 119 if self.buckets[index] != 0: 120 first_bucket_offset = index 121 break 122 for index in range(MAX_BUCKET_SIZE - 1, -1, -1): 123 if self.buckets[index] != 0: 124 last_bucket_offset = index 125 break 126 histogram.firstBucketOffset = first_bucket_offset 127 histogram.bucketCounts = [] 128 for index in range(first_bucket_offset, last_bucket_offset + 1): 129 histogram.bucketCounts.append(self.buckets[index]) 130 131 cpdef bint add_inputs_for_test(self, elements) except -1: 132 """Used for performance microbenchmark. 133 134 During runtime, add_input will be called through c-call, so we want to have 135 the same calling routine when running microbenchmark as application runtime. 136 Directly calling cpdef from def will cause significant overhead. 137 """ 138 for element in elements: 139 self.add_input(element) 140 141 cpdef int64_t calculate_bucket_index(self, int64_t element): 142 """Used for unit tests. 143 144 cdef calculate_bucket_index cannot be called directly from def. 145 """ 146 return self._fast_calculate_bucket_index(element) 147 148 cpdef tuple extract_output(self): 149 mean = self.sum // self.count if self.count else float('nan') 150 return mean, self.sum, self.count, self.min, self.max 151 152 cpdef merge(self, accumulators): 153 raise NotImplementedError()