github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/examples/snippets/transforms/aggregation/combineglobally.py (about) 1 # coding=utf-8 2 # 3 # Licensed to the Apache Software Foundation (ASF) under one or more 4 # contributor license agreements. See the NOTICE file distributed with 5 # this work for additional information regarding copyright ownership. 6 # The ASF licenses this file to You under the Apache License, Version 2.0 7 # (the "License"); you may not use this file except in compliance with 8 # the License. You may obtain a copy of the License at 9 # 10 # http://www.apache.org/licenses/LICENSE-2.0 11 # 12 # Unless required by applicable law or agreed to in writing, software 13 # distributed under the License is distributed on an "AS IS" BASIS, 14 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 # See the License for the specific language governing permissions and 16 # limitations under the License. 17 # 18 19 # pytype: skip-file 20 21 22 def combineglobally_function(test=None): 23 # [START combineglobally_function] 24 import apache_beam as beam 25 26 def get_common_items(sets): 27 # set.intersection() takes multiple sets as separete arguments. 28 # We unpack the `sets` list into multiple arguments with the * operator. 29 # The combine transform might give us an empty list of `sets`, 30 # so we use a list with an empty set as a default value. 31 return set.intersection(*(sets or [set()])) 32 33 with beam.Pipeline() as pipeline: 34 common_items = ( 35 pipeline 36 | 'Create produce' >> beam.Create([ 37 {'🍓', '🥕', '🍌', '🍅', '🌶️'}, 38 {'🍇', '🥕', '🥝', '🍅', '🥔'}, 39 {'🍉', '🥕', '🍆', '🍅', '🍍'}, 40 {'🥑', '🥕', '🌽', '🍅', '🥥'}, 41 ]) 42 | 'Get common items' >> beam.CombineGlobally(get_common_items) 43 | beam.Map(print)) 44 # [END combineglobally_function] 45 if test: 46 test(common_items) 47 48 49 def combineglobally_lambda(test=None): 50 # [START combineglobally_lambda] 51 import apache_beam as beam 52 53 with beam.Pipeline() as pipeline: 54 common_items = ( 55 pipeline 56 | 'Create produce' >> beam.Create([ 57 {'🍓', '🥕', '🍌', '🍅', '🌶️'}, 58 {'🍇', '🥕', '🥝', '🍅', '🥔'}, 59 {'🍉', '🥕', '🍆', '🍅', '🍍'}, 60 {'🥑', '🥕', '🌽', '🍅', '🥥'}, 61 ]) 62 | 'Get common items' >> 63 beam.CombineGlobally(lambda sets: set.intersection(*(sets or [set()]))) 64 | beam.Map(print)) 65 # [END combineglobally_lambda] 66 if test: 67 test(common_items) 68 69 70 def combineglobally_multiple_arguments(test=None): 71 # [START combineglobally_multiple_arguments] 72 import apache_beam as beam 73 74 with beam.Pipeline() as pipeline: 75 common_items_with_exceptions = ( 76 pipeline 77 | 'Create produce' >> beam.Create([ 78 {'🍓', '🥕', '🍌', '🍅', '🌶️'}, 79 {'🍇', '🥕', '🥝', '🍅', '🥔'}, 80 {'🍉', '🥕', '🍆', '🍅', '🍍'}, 81 {'🥑', '🥕', '🌽', '🍅', '🥥'}, 82 ]) 83 | 'Get common items with exceptions' >> beam.CombineGlobally( 84 lambda sets, exclude: \ 85 set.intersection(*(sets or [set()])) - exclude, 86 exclude={'🥕'}) 87 | beam.Map(print) 88 ) 89 # [END combineglobally_multiple_arguments] 90 if test: 91 test(common_items_with_exceptions) 92 93 94 def combineglobally_side_inputs_singleton(test=None): 95 # [START combineglobally_side_inputs_singleton] 96 import apache_beam as beam 97 98 with beam.Pipeline() as pipeline: 99 single_exclude = pipeline | 'Create single_exclude' >> beam.Create(['🥕']) 100 101 common_items_with_exceptions = ( 102 pipeline 103 | 'Create produce' >> beam.Create([ 104 {'🍓', '🥕', '🍌', '🍅', '🌶️'}, 105 {'🍇', '🥕', '🥝', '🍅', '🥔'}, 106 {'🍉', '🥕', '🍆', '🍅', '🍍'}, 107 {'🥑', '🥕', '🌽', '🍅', '🥥'}, 108 ]) 109 | 'Get common items with exceptions' >> beam.CombineGlobally( 110 lambda sets, single_exclude: \ 111 set.intersection(*(sets or [set()])) - {single_exclude}, 112 single_exclude=beam.pvalue.AsSingleton(single_exclude)) 113 | beam.Map(print) 114 ) 115 # [END combineglobally_side_inputs_singleton] 116 if test: 117 test(common_items_with_exceptions) 118 119 120 def combineglobally_side_inputs_iter(test=None): 121 # [START combineglobally_side_inputs_iter] 122 import apache_beam as beam 123 124 with beam.Pipeline() as pipeline: 125 exclude = pipeline | 'Create exclude' >> beam.Create(['🥕']) 126 127 common_items_with_exceptions = ( 128 pipeline 129 | 'Create produce' >> beam.Create([ 130 {'🍓', '🥕', '🍌', '🍅', '🌶️'}, 131 {'🍇', '🥕', '🥝', '🍅', '🥔'}, 132 {'🍉', '🥕', '🍆', '🍅', '🍍'}, 133 {'🥑', '🥕', '🌽', '🍅', '🥥'}, 134 ]) 135 | 'Get common items with exceptions' >> beam.CombineGlobally( 136 lambda sets, exclude: \ 137 set.intersection(*(sets or [set()])) - set(exclude), 138 exclude=beam.pvalue.AsIter(exclude)) 139 | beam.Map(print) 140 ) 141 # [END combineglobally_side_inputs_iter] 142 if test: 143 test(common_items_with_exceptions) 144 145 146 def combineglobally_side_inputs_dict(test=None): 147 # [START combineglobally_side_inputs_dict] 148 import apache_beam as beam 149 150 def get_custom_common_items(sets, options): 151 sets = sets or [set()] 152 common_items = set.intersection(*sets) 153 common_items |= options['include'] # union 154 common_items &= options['exclude'] # intersection 155 return common_items 156 157 with beam.Pipeline() as pipeline: 158 options = pipeline | 'Create options' >> beam.Create([ 159 ('exclude', {'🥕'}), 160 ('include', {'🍇', '🌽'}), 161 ]) 162 163 custom_common_items = ( 164 pipeline 165 | 'Create produce' >> beam.Create([ 166 {'🍓', '🥕', '🍌', '🍅', '🌶️'}, 167 {'🍇', '🥕', '🥝', '🍅', '🥔'}, 168 {'🍉', '🥕', '🍆', '🍅', '🍍'}, 169 {'🥑', '🥕', '🌽', '🍅', '🥥'}, 170 ]) 171 | 'Get common items' >> beam.CombineGlobally( 172 get_custom_common_items, options=beam.pvalue.AsDict(options)) 173 | beam.Map(print)) 174 # [END combineglobally_side_inputs_dict] 175 if test: 176 test(custom_common_items) 177 178 179 def combineglobally_combinefn(test=None): 180 # [START combineglobally_combinefn] 181 import apache_beam as beam 182 183 class PercentagesFn(beam.CombineFn): 184 def create_accumulator(self): 185 return {} 186 187 def add_input(self, accumulator, input): 188 # accumulator == {} 189 # input == '🥕' 190 if input not in accumulator: 191 accumulator[input] = 0 # {'🥕': 0} 192 accumulator[input] += 1 # {'🥕': 1} 193 return accumulator 194 195 def merge_accumulators(self, accumulators): 196 # accumulators == [ 197 # {'🥕': 1, '🍅': 2}, 198 # {'🥕': 1, '🍅': 1, '🍆': 1}, 199 # {'🥕': 1, '🍅': 3}, 200 # ] 201 merged = {} 202 for accum in accumulators: 203 for item, count in accum.items(): 204 if item not in merged: 205 merged[item] = 0 206 merged[item] += count 207 # merged == {'🥕': 3, '🍅': 6, '🍆': 1} 208 return merged 209 210 def extract_output(self, accumulator): 211 # accumulator == {'🥕': 3, '🍅': 6, '🍆': 1} 212 total = sum(accumulator.values()) # 10 213 percentages = {item: count / total for item, count in accumulator.items()} 214 # percentages == {'🥕': 0.3, '🍅': 0.6, '🍆': 0.1} 215 return percentages 216 217 with beam.Pipeline() as pipeline: 218 percentages = ( 219 pipeline 220 | 'Create produce' >> beam.Create( 221 ['🥕', '🍅', '🍅', '🥕', '🍆', '🍅', '🍅', '🍅', '🥕', '🍅']) 222 | 'Get percentages' >> beam.CombineGlobally(PercentagesFn()) 223 | beam.Map(print)) 224 # [END combineglobally_combinefn] 225 if test: 226 test(percentages)