github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/examples/cookbook/group_with_coder.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 """An example of using custom classes and coder for grouping operations. 19 20 This workflow demonstrates registration and usage of a custom coder for a user- 21 defined class. A deterministic custom coder is needed to use a class as a key in 22 a combine or group operation. 23 24 This example assumes an input file with, on each line, a comma-separated name 25 and score. 26 """ 27 28 # pytype: skip-file 29 30 import argparse 31 import logging 32 import sys 33 import typing 34 35 import apache_beam as beam 36 from apache_beam import coders 37 from apache_beam.io import ReadFromText 38 from apache_beam.io import WriteToText 39 from apache_beam.options.pipeline_options import PipelineOptions 40 from apache_beam.options.pipeline_options import SetupOptions 41 from apache_beam.typehints.decorators import with_output_types 42 43 44 class Player(object): 45 """A custom class used as a key in combine/group transforms.""" 46 def __init__(self, name): 47 self.name = name 48 49 50 class PlayerCoder(coders.Coder): 51 """A custom coder for the Player class.""" 52 def encode(self, o): 53 """Encode to bytes with a trace that coder was used.""" 54 # Our encoding prepends an 'x:' prefix. 55 return b'x:%s' % str(o.name).encode('utf-8') 56 57 def decode(self, s): 58 # To decode, we strip off the prepended 'x:' prefix. 59 s = s.decode('utf-8') 60 assert s[0:2] == 'x:' 61 return Player(s[2:]) 62 63 def is_deterministic(self): 64 # Since coded Player objects are used as keys below with 65 # beam.CombinePerKey(sum), we require that this coder is deterministic 66 # (i.e., two equivalent instances of the classes are encoded into the same 67 # byte string) in order to guarantee consistent results. 68 return True 69 70 71 # Annotate the get_players function so that the typehint system knows that the 72 # input to the CombinePerKey operation is a key-value pair of a Player object 73 # and an integer. 74 @with_output_types(typing.Tuple[Player, int]) 75 def get_players(descriptor): 76 name, points = descriptor.split(',') 77 return Player(name), int(points) 78 79 80 def run(args=None, save_main_session=True): 81 """Runs the workflow computing total points from a collection of matches.""" 82 83 if args is None: 84 args = sys.argv[1:] 85 parser = argparse.ArgumentParser() 86 parser.add_argument('--input', required=True, help='Input file to process.') 87 parser.add_argument( 88 '--output', required=True, help='Output file to write results to.') 89 known_args, pipeline_args = parser.parse_known_args(args) 90 # We use the save_main_session option because one or more DoFn's in this 91 # workflow rely on global context (e.g., a module imported at module level). 92 pipeline_options = PipelineOptions(pipeline_args) 93 pipeline_options.view_as(SetupOptions).save_main_session = save_main_session 94 with beam.Pipeline(options=pipeline_options) as p: 95 96 # Register the custom coder for the Player class, so that it will be used in 97 # the computation. 98 coders.registry.register_coder(Player, PlayerCoder) 99 100 ( # pylint: disable=expression-not-assigned 101 p 102 | ReadFromText(known_args.input) 103 # The get_players function is annotated with a type hint above, 104 # so the type system knows the output type of the following operation 105 # is a key-value pair of a Player and an int. Please see the 106 # documentation for details on types that are inferred automatically 107 # as well as other ways to specify type hints. 108 | beam.Map(get_players) 109 # The output type hint of the previous step is used to infer that the 110 # key type of the following operation is the Player type. Since a 111 # custom coder is registered for the Player class above, 112 # a PlayerCoder will be used to encode Player objects as keys for this 113 # combine operation. 114 | beam.CombinePerKey(sum) 115 | beam.Map(lambda k_v: '%s,%d' % (k_v[0].name, k_v[1])) 116 | WriteToText(known_args.output)) 117 118 119 if __name__ == '__main__': 120 logging.getLogger().setLevel(logging.INFO) 121 run()