github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/examples/cookbook/multiple_output_pardo.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 """A workflow demonstrating a DoFn with multiple outputs. 19 20 DoFns may produce multiple outputs. Outputs that are not the default ("main") 21 output are marked with a tag at output time and later the same tag will be used 22 to get the corresponding result (a PCollection) for that output. 23 24 This is a slightly modified version of the basic wordcount example. In this 25 example words are divided into 2 buckets as shorts words (3 characters in length 26 or less) and words (all other words). There will be 3 output files::: 27 28 [OUTPUT]-chars : Character count for the input. 29 [OUTPUT]-short-words : Word count for short words only. 30 [OUTPUT]-words : Word count for all other words. 31 32 To execute this pipeline locally, specify a local output file or output prefix 33 on GCS::: 34 35 --output [YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX] 36 37 To execute this pipeline using the Google Cloud Dataflow service, specify 38 pipeline configuration::: 39 40 --project YOUR_PROJECT_ID 41 --region GCE_REGION 42 --staging_location gs://YOUR_STAGING_DIRECTORY 43 --temp_location gs://YOUR_TEMP_DIRECTORY 44 --job_name YOUR_JOB_NAME 45 --runner DataflowRunner 46 47 and an output prefix on GCS::: 48 49 --output gs://YOUR_OUTPUT_PREFIX 50 """ 51 52 # pytype: skip-file 53 54 # beam-playground: 55 # name: MultipleOutputPardo 56 # description: This is a slightly modified version 57 # of the basic wordcount example. In this example words 58 # are divided into 2 buckets as shorts 59 # words (3 characters in length or less) and words (other). 60 # multifile: false 61 # pipeline_options: --output output.txt 62 # context_line: 80 63 # categories: 64 # - IO 65 # - Options 66 # - Multiple Outputs 67 # complexity: MEDIUM 68 # tags: 69 # - count 70 # - split 71 # - strings 72 73 import argparse 74 import logging 75 import re 76 77 import apache_beam as beam 78 from apache_beam import pvalue 79 from apache_beam.io import ReadFromText 80 from apache_beam.io import WriteToText 81 from apache_beam.options.pipeline_options import PipelineOptions 82 from apache_beam.options.pipeline_options import SetupOptions 83 84 85 class SplitLinesToWordsFn(beam.DoFn): 86 """A transform to split a line of text into individual words. 87 88 This transform will have 3 outputs: 89 - main output: all words that are longer than 3 characters. 90 - short words output: all other words. 91 - character count output: Number of characters in each processed line. 92 """ 93 94 # These tags will be used to tag the outputs of this DoFn. 95 OUTPUT_TAG_SHORT_WORDS = 'tag_short_words' 96 OUTPUT_TAG_CHARACTER_COUNT = 'tag_character_count' 97 98 def process(self, element): 99 """Receives a single element (a line) and produces words and character 100 counts. 101 102 Important things to note here: 103 - For a single element you may produce multiple main outputs: 104 words of a single line. 105 - For that same input you may produce multiple outputs, potentially 106 across multiple PCollections 107 - Outputs may have different types (count) or may share the same type 108 (words) as with the main output. 109 110 Args: 111 element: processing element. 112 113 Yields: 114 words as main output, short words as tagged output, line character count 115 as tagged output. 116 """ 117 # yield a count (integer) to the OUTPUT_TAG_CHARACTER_COUNT tagged 118 # collection. 119 yield pvalue.TaggedOutput(self.OUTPUT_TAG_CHARACTER_COUNT, len(element)) 120 121 words = re.findall(r'[A-Za-z\']+', element) 122 for word in words: 123 if len(word) <= 3: 124 # yield word as an output to the OUTPUT_TAG_SHORT_WORDS tagged 125 # collection. 126 yield pvalue.TaggedOutput(self.OUTPUT_TAG_SHORT_WORDS, word) 127 else: 128 # yield word to add it to the main collection. 129 yield word 130 131 132 class CountWords(beam.PTransform): 133 """A transform to count the occurrences of each word. 134 135 A PTransform that converts a PCollection containing words into a PCollection 136 of "word: count" strings. 137 """ 138 def expand(self, pcoll): 139 def count_ones(word_ones): 140 (word, ones) = word_ones 141 return (word, sum(ones)) 142 143 def format_result(word_count): 144 (word, count) = word_count 145 return '%s: %s' % (word, count) 146 147 return ( 148 pcoll 149 | 'pair_with_one' >> beam.Map(lambda x: (x, 1)) 150 | 'group' >> beam.GroupByKey() 151 | 'count' >> beam.Map(count_ones) 152 | 'format' >> beam.Map(format_result)) 153 154 155 def run(argv=None, save_main_session=True): 156 """Runs the workflow counting the long words and short words separately.""" 157 158 parser = argparse.ArgumentParser() 159 parser.add_argument( 160 '--input', 161 default='gs://dataflow-samples/shakespeare/kinglear.txt', 162 help='Input file to process.') 163 parser.add_argument( 164 '--output', 165 required=True, 166 help='Output prefix for files to write results to.') 167 known_args, pipeline_args = parser.parse_known_args(argv) 168 # We use the save_main_session option because one or more DoFn's in this 169 # workflow rely on global context (e.g., a module imported at module level). 170 pipeline_options = PipelineOptions(pipeline_args) 171 pipeline_options.view_as(SetupOptions).save_main_session = save_main_session 172 with beam.Pipeline(options=pipeline_options) as p: 173 174 lines = p | ReadFromText(known_args.input) 175 176 # with_outputs allows accessing the explicitly tagged outputs of a DoFn. 177 split_lines_result = ( 178 lines 179 | beam.ParDo(SplitLinesToWordsFn()).with_outputs( 180 SplitLinesToWordsFn.OUTPUT_TAG_SHORT_WORDS, 181 SplitLinesToWordsFn.OUTPUT_TAG_CHARACTER_COUNT, 182 main='words')) 183 184 # split_lines_result is an object of type DoOutputsTuple. It supports 185 # accessing result in alternative ways. 186 words, _, _ = split_lines_result 187 short_words = split_lines_result[SplitLinesToWordsFn.OUTPUT_TAG_SHORT_WORDS] 188 character_count = split_lines_result.tag_character_count 189 190 # pylint: disable=expression-not-assigned 191 ( 192 character_count 193 | 'pair_with_key' >> beam.Map(lambda x: ('chars_temp_key', x)) 194 | beam.GroupByKey() 195 | 'count chars' >> beam.Map(lambda char_counts: sum(char_counts[1])) 196 | 'write chars' >> WriteToText(known_args.output + '-chars')) 197 198 # pylint: disable=expression-not-assigned 199 ( 200 short_words 201 | 'count short words' >> CountWords() 202 | 203 'write short words' >> WriteToText(known_args.output + '-short-words')) 204 205 # pylint: disable=expression-not-assigned 206 ( 207 words 208 | 'count words' >> CountWords() 209 | 'write words' >> WriteToText(known_args.output + '-words')) 210 211 212 if __name__ == '__main__': 213 logging.getLogger().setLevel(logging.INFO) 214 run()