github.com/apache/beam/sdks/v2@v2.48.2/go/examples/multiout/multiout.go (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  // multiout is a wordcount variation that uses a multi-outout DoFn
    17  // and writes 2 output files.
    18  package main
    19  
    20  // beam-playground:
    21  //   name: MultiOut
    22  //   description: An example that counts words in Shakespeare's works and writes 2 output files,
    23  //     -- big - for small words,
    24  //     -- small - for big words.
    25  //   multifile: false
    26  //   pipeline_options: --small sOutput.txt --big bOutput.txt
    27  //   context_line: 53
    28  //   categories:
    29  //     - IO
    30  //     - Options
    31  //     - Branching
    32  //     - Multiple Outputs
    33  //   complexity: MEDIUM
    34  //   tags:
    35  //     - count
    36  //     - io
    37  //     - strings
    38  
    39  import (
    40  	"context"
    41  	"flag"
    42  	"fmt"
    43  	"log"
    44  	"regexp"
    45  
    46  	"github.com/apache/beam/sdks/v2/go/pkg/beam"
    47  	"github.com/apache/beam/sdks/v2/go/pkg/beam/io/textio"
    48  	"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
    49  	"github.com/apache/beam/sdks/v2/go/pkg/beam/transforms/stats"
    50  	"github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx"
    51  )
    52  
    53  var (
    54  	input = flag.String("input", "gs://apache-beam-samples/shakespeare/kinglear.txt", "File(s) to read.")
    55  	small = flag.String("small", "", "Output file for small words (required).")
    56  	big   = flag.String("big", "", "Output file for big words (required).")
    57  )
    58  
    59  func init() {
    60  	register.Function3x0(splitFn)
    61  	register.Function2x1(formatFn)
    62  	register.Emitter1[string]()
    63  }
    64  
    65  var wordRE = regexp.MustCompile(`[a-zA-Z]+('[a-z])?`)
    66  
    67  func splitFn(line string, big, small func(string)) {
    68  	for _, word := range wordRE.FindAllString(line, -1) {
    69  		if len(word) > 5 {
    70  			big(word)
    71  		} else {
    72  			small(word)
    73  		}
    74  	}
    75  }
    76  
    77  func formatFn(w string, c int) string {
    78  	return fmt.Sprintf("%s: %v", w, c)
    79  }
    80  
    81  func writeCounts(s beam.Scope, col beam.PCollection, filename string) {
    82  	counted := stats.Count(s, col)
    83  	textio.Write(s, filename, beam.ParDo(s, formatFn, counted))
    84  }
    85  
    86  func main() {
    87  	flag.Parse()
    88  	beam.Init()
    89  
    90  	if *small == "" || *big == "" {
    91  		log.Fatal("No outputs provided")
    92  	}
    93  
    94  	p := beam.NewPipeline()
    95  	s := p.Root()
    96  
    97  	lines := textio.Read(s, *input)
    98  	bcol, scol := beam.ParDo2(s, splitFn, lines)
    99  	writeCounts(s.Scope("Big"), bcol, *big)
   100  	writeCounts(s.Scope("Small"), scol, *small)
   101  
   102  	if err := beamx.Run(context.Background(), p); err != nil {
   103  		log.Fatalf("Failed to execute job: %v", err)
   104  	}
   105  }