github.com/apache/beam/sdks/v2@v2.48.2/go/test/regression/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 regression contains pipeline regression tests. 17 package regression 18 19 import ( 20 "github.com/apache/beam/sdks/v2/go/pkg/beam" 21 "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert" 22 "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest" 23 ) 24 25 func directFn(elm int) int { 26 return elm + 1 27 } 28 29 // DirectParDo tests direct form output DoFns. 30 func DirectParDo() *beam.Pipeline { 31 p, s := beam.NewPipelineWithRoot() 32 33 direct := beam.ParDo(s, directFn, beam.Create(s, 1, 2, 3)) 34 passert.Sum(s, direct, "direct", 3, 9) 35 36 return p 37 } 38 39 func emitFn(elm int, emit func(int)) { 40 emit(elm + 1) 41 } 42 43 // EmitParDo tests emit form output DoFns. 44 func EmitParDo() *beam.Pipeline { 45 p, s := beam.NewPipelineWithRoot() 46 47 emit := beam.ParDo(s, emitFn, beam.Create(s, 1, 2, 3)) 48 passert.Sum(s, emit, "emit", 3, 9) 49 50 return p 51 } 52 53 func emit2Fn(elm int, emit, emit2 func(int)) { 54 emit(elm + 1) 55 emit2(elm + 2) 56 } 57 58 // MultiEmitParDo tests double emit form output DoFns. 59 func MultiEmitParDo() *beam.Pipeline { 60 p, s := beam.NewPipelineWithRoot() 61 62 emit1, emit2 := beam.ParDo2(s, emit2Fn, beam.Create(s, 1, 2, 3)) 63 passert.Sum(s, emit1, "emit2_1", 3, 9) 64 passert.Sum(s, emit2, "emit2_2", 3, 12) 65 66 return p 67 } 68 69 func mixedFn(elm int, emit func(int)) int { 70 emit(elm + 2) 71 return elm + 1 72 } 73 74 // MixedOutputParDo tests mixed direct + emit form output DoFns. 75 func MixedOutputParDo() *beam.Pipeline { 76 p, s := beam.NewPipelineWithRoot() 77 78 mixed1, mixed2 := beam.ParDo2(s, mixedFn, beam.Create(s, 1, 2, 3)) 79 passert.Sum(s, mixed1, "mixed_1", 3, 9) 80 passert.Sum(s, mixed2, "mixed_2", 3, 12) 81 82 return p 83 } 84 85 func directCountFn(_ int, values func(*int) bool) (int, error) { 86 sum := 0 87 var i int 88 for values(&i) { 89 sum += i 90 } 91 return sum, nil 92 } 93 94 // DirectParDoAfterGBK generates a pipeline with a direct-form 95 // ParDo after a GBK. See: BEAM-3978 and BEAM-4175. 96 func DirectParDoAfterGBK() *beam.Pipeline { 97 p, s, col := ptest.Create([]any{1, 2, 3, 4}) 98 99 keyed := beam.GroupByKey(s, beam.AddFixedKey(s, col)) 100 sum := beam.ParDo(s, directCountFn, keyed) 101 passert.Equals(s, beam.DropKey(s, beam.AddFixedKey(s, sum)), 10) 102 103 return p 104 } 105 106 func emitCountFn(_ int, values func(*int) bool, emit func(int)) error { 107 sum := 0 108 var i int 109 for values(&i) { 110 sum += i 111 } 112 emit(sum) 113 return nil 114 } 115 116 // EmitParDoAfterGBK generates a pipeline with a emit-form 117 // ParDo after a GBK. See: BEAM-3978 and BEAM-4175. 118 func EmitParDoAfterGBK() *beam.Pipeline { 119 p, s, col := ptest.Create([]any{1, 2, 3, 4}) 120 121 keyed := beam.GroupByKey(s, beam.AddFixedKey(s, col)) 122 sum := beam.ParDo(s, emitCountFn, keyed) 123 passert.Equals(s, beam.DropKey(s, beam.AddFixedKey(s, sum)), 10) 124 125 return p 126 }