github.com/apache/beam/sdks/v2@v2.48.2/go/test/integration/primitives/cogbk.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 contains integration tests for primitives in beam. 17 package primitives 18 19 import ( 20 "fmt" 21 22 "github.com/apache/beam/sdks/v2/go/pkg/beam" 23 "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert" 24 ) 25 26 func genA(_ []byte, emit func(string, int)) { 27 emit("a", 1) 28 emit("a", 2) 29 emit("a", 3) 30 emit("b", 4) 31 emit("b", 5) 32 emit("c", 6) 33 } 34 35 func genB(_ []byte, emit func(string, int)) { 36 emit("a", 7) 37 emit("b", 8) 38 emit("d", 9) 39 } 40 41 func genC(_ []byte, emit func(string, string)) { 42 emit("a", "alpha") 43 emit("c", "charlie") 44 emit("d", "delta") 45 } 46 47 func genD(_ []byte, emit func(string, int)) { 48 emit("a", 1) 49 emit("a", 1) 50 emit("a", 1) 51 emit("b", 4) 52 emit("b", 4) 53 emit("c", 6) 54 emit("c", 6) 55 emit("c", 6) 56 emit("c", 6) 57 emit("c", 6) 58 } 59 60 func shortFn(_ string, ds func(*int) bool, emit func(int)) { 61 var v int 62 ds(&v) 63 emit(v) 64 } 65 66 func sum(nums func(*int) bool) int { 67 var ret, i int 68 for nums(&i) { 69 ret += i 70 } 71 return ret 72 } 73 74 func lenSum(strings func(*string) bool) int { 75 var ret int 76 var s string 77 for strings(&s) { 78 ret += len(s) 79 } 80 return ret 81 } 82 83 func joinFn(key string, as, bs func(*int) bool, cs func(*string) bool, emit func(string, int)) { 84 emit(key, sum(as)+sum(bs)+lenSum(cs)) 85 } 86 87 func splitFn(key string, v int, a, b, c, d func(int)) { 88 switch key { 89 case "a": 90 a(v) 91 case "b": 92 b(v) 93 case "c": 94 c(v) 95 case "d": 96 d(v) 97 default: 98 panic(fmt.Sprintf("bad key: %v", key)) 99 } 100 } 101 102 // CoGBK tests CoGBK. 103 func CoGBK() *beam.Pipeline { 104 p, s := beam.NewPipelineWithRoot() 105 106 s2 := s.Scope("SubScope") 107 as := beam.ParDo(s2, genA, beam.Impulse(s)) 108 bs := beam.ParDo(s2, genB, beam.Impulse(s)) 109 cs := beam.ParDo(s2, genC, beam.Impulse(s)) 110 grouped := beam.CoGroupByKey(s2, as, bs, cs) 111 joined := beam.ParDo(s2, joinFn, grouped) 112 113 a, b, c, d := beam.ParDo4(s, splitFn, joined) 114 115 passert.Sum(s, a, "a", 1, 18) 116 passert.Sum(s, b, "b", 1, 17) 117 passert.Sum(s, c, "c", 1, 13) 118 passert.Sum(s, d, "d", 1, 14) 119 120 return p 121 } 122 123 // GBKShortRead tests GBK with a short read on the iterator. 124 func GBKShortRead() *beam.Pipeline { 125 p, s := beam.NewPipelineWithRoot() 126 127 ds := beam.ParDo(s, genD, beam.Impulse(s)) 128 grouped := beam.GroupByKey(s, ds) 129 short := beam.ParDo(s, shortFn, grouped) 130 131 passert.Sum(s, short, "shorted", 3, 11) 132 133 return p 134 } 135 136 // Reshuffle tests Reshuffle. 137 func Reshuffle() *beam.Pipeline { 138 p, s := beam.NewPipelineWithRoot() 139 140 in := beam.Create(s, 1, 2, 3, 4, 5, 6, 7, 8, 9) 141 in = beam.Reshuffle(s, in) 142 passert.Sum(s, in, "reshuffled", 9, 45) 143 144 return p 145 } 146 147 // ReshuffleKV tests Reshuffle with KV PCollections. 148 func ReshuffleKV() *beam.Pipeline { 149 p, s := beam.NewPipelineWithRoot() 150 151 s2 := s.Scope("SubScope") 152 as := beam.ParDo(s2, genA, beam.Impulse(s)) 153 bs := beam.ParDo(s2, genB, beam.Impulse(s)) 154 cs := beam.ParDo(s2, genC, beam.Impulse(s)) 155 156 as = beam.Reshuffle(s2, as) 157 cs = beam.Reshuffle(s2, cs) 158 159 grouped := beam.CoGroupByKey(s2, as, bs, cs) 160 joined := beam.ParDo(s2, joinFn, grouped) 161 162 joined = beam.Reshuffle(s, joined) 163 164 a, b, c, d := beam.ParDo4(s, splitFn, joined) 165 166 passert.Sum(s, a, "a", 1, 18) 167 passert.Sum(s, b, "b", 1, 17) 168 passert.Sum(s, c, "c", 1, 13) 169 passert.Sum(s, d, "d", 1, 14) 170 171 return p 172 }