github.com/apache/beam/sdks/v2@v2.48.2/go/test/integration/primitives/pardo.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 package primitives 17 18 import ( 19 "flag" 20 "fmt" 21 22 "github.com/apache/beam/sdks/v2/go/pkg/beam" 23 "github.com/apache/beam/sdks/v2/go/pkg/beam/register" 24 "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert" 25 ) 26 27 func init() { 28 register.Function4x0(emit3Fn) 29 register.Function2x1(sumValuesFn) 30 register.Function2x1(sumKVValuesFn) 31 register.Function1x2(splitStringPair) 32 register.Function3x2(asymJoinFn) 33 register.Function5x0(splitByName) 34 35 register.Iter1[int]() 36 register.Iter2[int, int]() 37 register.Emitter1[string]() 38 register.Emitter1[int]() 39 } 40 41 func emit3Fn(elm int, emit, emit2, emit3 func(int)) { 42 emit(elm + 1) 43 emit2(elm + 2) 44 emit3(elm + 3) 45 } 46 47 // ParDoMultiOutput test a DoFn with multiple output. 48 func ParDoMultiOutput() *beam.Pipeline { 49 p, s := beam.NewPipelineWithRoot() 50 51 in := beam.Create(s, 1) 52 emit1, emit2, emit3 := beam.ParDo3(s, emit3Fn, in) 53 passert.Sum(s, emit1, "emit1", 1, 2) 54 passert.Sum(s, emit2, "emit2", 1, 3) 55 passert.Sum(s, emit3, "emit3", 1, 4) 56 57 return p 58 } 59 60 func sumValuesFn(_ []byte, values func(*int) bool) int { 61 sum := 0 62 var i int 63 for values(&i) { 64 sum += i 65 } 66 return sum 67 } 68 69 // ParDoSideInput computes the sum of ints using a side input. 70 func ParDoSideInput() *beam.Pipeline { 71 p, s := beam.NewPipelineWithRoot() 72 73 in := beam.Create(s, 1, 2, 3, 4, 5, 6, 7, 8, 9) 74 sub := s.Scope("subscope") // Ensure scoping works with side inputs. See: BEAM-5354 75 out := beam.ParDo(sub, sumValuesFn, beam.Impulse(s), beam.SideInput{Input: in}) 76 passert.Sum(s, out, "out", 1, 45) 77 78 return p 79 } 80 81 func sumKVValuesFn(_ []byte, values func(*int, *int) bool) int { 82 sum := 0 83 var i, k int 84 for values(&i, &k) { 85 sum += i 86 sum += k 87 } 88 return sum 89 } 90 91 // ParDoKVSideInput computes the sum of ints using a KV side input. 92 func ParDoKVSideInput() *beam.Pipeline { 93 p, s := beam.NewPipelineWithRoot() 94 95 in := beam.Create(s, 1, 2, 3, 4, 5, 6, 7, 8, 9) 96 kv := beam.AddFixedKey(s, in) // i -> (0,i) 97 out := beam.ParDo(s, sumKVValuesFn, beam.Impulse(s), beam.SideInput{Input: kv}) 98 passert.Sum(s, out, "out", 1, 45) 99 100 return p 101 } 102 103 type stringPair struct { 104 K, V string 105 } 106 107 func splitStringPair(e stringPair) (string, string) { 108 return e.K, e.V 109 } 110 111 var emailSlice = []stringPair{ 112 {"amy", "amy@example.com"}, 113 {"james", "james@email.com"}, 114 {"carl", "carl@example.com"}, 115 {"julia", "julia@example.com"}, 116 {"carl", "carl@email.com"}, 117 {"james", "james@example.com"}, 118 } 119 120 var phoneSlice = []stringPair{ 121 {"amy", "111-222-3333"}, 122 {"james", "222-333-4444"}, 123 } 124 125 // CreateAndSplit makes a KV PCollection from a list of stringPair types 126 func CreateAndSplit(s beam.Scope, input []stringPair) beam.PCollection { 127 initial := beam.CreateList(s, input) 128 return beam.ParDo(s, splitStringPair, initial) 129 } 130 131 // ParDoMultiMapSideInput checks that the multimap side input access pattern 132 // works correctly, properly producing the correct output with an asymmetric join. 133 func ParDoMultiMapSideInput() *beam.Pipeline { 134 beam.Init() 135 p, s := beam.NewPipelineWithRoot() 136 emailsKV := CreateAndSplit(s.Scope("CreateEmails"), emailSlice) 137 phonesKV := CreateAndSplit(s.Scope("CreatePhones"), phoneSlice) 138 output := beam.ParDo(s, asymJoinFn, phonesKV, beam.SideInput{Input: emailsKV}) 139 passert.Count(s, output, "post-join", 2) 140 amyOut, jamesOut, noMatch := beam.ParDo3(s, splitByName, output) 141 passert.Equals(s, amyOut, "amy@example.com", "111-222-3333") 142 passert.Equals(s, jamesOut, "james@email.com", "james@example.com", "222-333-4444") 143 passert.Empty(s, noMatch) 144 return p 145 } 146 147 func asymJoinFn(k, v string, mapSide func(string) func(*string) bool) (string, []string) { 148 var out string 149 results := []string{v} 150 iter := mapSide(k) 151 for iter(&out) { 152 results = append(results, out) 153 } 154 return k, results 155 } 156 157 func splitByName(key string, vals []string, a, j, d func(string)) { 158 var emitter func(string) 159 switch key { 160 case "amy": 161 emitter = a 162 case "james": 163 emitter = j 164 default: 165 emitter = d 166 } 167 for _, val := range vals { 168 emitter(val) 169 } 170 } 171 172 // ParDoPipelineOptions creates a pipeline with flag options to validate 173 // that a DoFn can access them as PipelineOptions. 174 func ParDoPipelineOptions() *beam.Pipeline { 175 // Setup some fake flags 176 flag.String("A", "", "Flag for testing.") 177 flag.String("B", "", "Flag for testing.") 178 flag.String("C", "", "Flag for testing.") 179 flag.CommandLine.Parse([]string{"--A=123", "--B=456", "--C=789"}) 180 181 p, s := beam.NewPipelineWithRoot() 182 183 emitted := beam.ParDo(s, emitPipelineOptions, beam.Impulse(s)) 184 passert.Equals(s, emitted, "A: 123", "B: 456", "C: 789") 185 186 return p 187 } 188 189 func emitPipelineOptions(_ []byte, emit func(string)) { 190 emit(fmt.Sprintf("%s: %s", "A", beam.PipelineOptions.Get("A"))) 191 emit(fmt.Sprintf("%s: %s", "B", beam.PipelineOptions.Get("B"))) 192 emit(fmt.Sprintf("%s: %s", "C", beam.PipelineOptions.Get("C"))) 193 }