github.com/apache/beam/sdks/v2@v2.48.2/go/examples/snippets/04transforms_test.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 snippets 17 18 import ( 19 "testing" 20 21 "github.com/apache/beam/sdks/v2/go/pkg/beam" 22 "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/passert" 23 "github.com/apache/beam/sdks/v2/go/pkg/beam/testing/ptest" 24 ) 25 26 func TestMain(m *testing.M) { 27 ptest.Main(m) 28 } 29 30 func TestParDo(t *testing.T) { 31 p, s, input := ptest.CreateList([]string{"one", "two", "three"}) 32 lens := applyWordLen(s, input) 33 passert.Equals(s, lens, 3, 3, 5) 34 ptest.RunAndValidate(t, p) 35 } 36 37 func TestParDo_anon(t *testing.T) { 38 p, s, input := ptest.CreateList([]string{"one", "two", "three"}) 39 lens := applyWordLenAnon(s, input) 40 passert.Equals(s, lens, 3, 3, 5) 41 ptest.RunAndValidate(t, p) 42 } 43 44 func TestFormatCoGBKResults(t *testing.T) { 45 // [START cogroupbykey_outputs] 46 // Synthetic example results of a cogbk. 47 results := []struct { 48 Key string 49 Emails, Phones []string 50 }{ 51 { 52 Key: "amy", 53 Emails: []string{"amy@example.com"}, 54 Phones: []string{"111-222-3333", "333-444-5555"}, 55 }, { 56 Key: "carl", 57 Emails: []string{"carl@email.com", "carl@example.com"}, 58 Phones: []string{"444-555-6666"}, 59 }, { 60 Key: "james", 61 Emails: []string{}, 62 Phones: []string{"222-333-4444"}, 63 }, { 64 Key: "julia", 65 Emails: []string{"julia@example.com"}, 66 Phones: []string{}, 67 }, 68 } 69 // [END cogroupbykey_outputs] 70 71 // [START cogroupbykey_formatted_outputs] 72 formattedResults := []string{ 73 "amy; ['amy@example.com']; ['111-222-3333', '333-444-5555']", 74 "carl; ['carl@email.com', 'carl@example.com']; ['444-555-6666']", 75 "james; []; ['222-333-4444']", 76 "julia; ['julia@example.com']; []", 77 } 78 // [END cogroupbykey_formatted_outputs] 79 80 // Helper to fake iterators for unit testing. 81 makeIter := func(vs []string) func(*string) bool { 82 i := 0 83 return func(v *string) bool { 84 if i >= len(vs) { 85 return false 86 } 87 *v = vs[i] 88 i++ 89 return true 90 } 91 } 92 93 for i, result := range results { 94 got := formatCoGBKResults(result.Key, makeIter(result.Emails), makeIter(result.Phones)) 95 want := formattedResults[i] 96 if got != want { 97 t.Errorf("%d.%v, got %q, want %q", i, result.Key, got, want) 98 } 99 } 100 101 p, s := beam.NewPipelineWithRoot() 102 formattedCoGBK := coGBKExample(s) 103 passert.Equals(s, formattedCoGBK, formattedResults[0], formattedResults[1], formattedResults[2], formattedResults[3]) 104 ptest.RunAndValidate(t, p) 105 } 106 107 func TestCombine(t *testing.T) { 108 p, s, input := ptest.CreateList([]int{1, 2, 3}) 109 avg := globallyAverage(s, input) 110 passert.Equals(s, avg, float64(2.0)) 111 ptest.RunAndValidate(t, p) 112 } 113 114 func TestCombineWithDefault_useDefault(t *testing.T) { 115 p, s, input := ptest.CreateList([]int{}) 116 avg := globallyAverageWithDefault(s, input) 117 passert.Equals(s, avg, float64(0)) 118 ptest.RunAndValidate(t, p) 119 } 120 121 func TestCombineWithDefault_useAverage(t *testing.T) { 122 p, s, input := ptest.CreateList([]int{1, 2, 3}) 123 avg := globallyAverageWithDefault(s, input) 124 passert.Equals(s, avg, float64(2.0)) 125 ptest.RunAndValidate(t, p) 126 } 127 128 func TestCombine_sum(t *testing.T) { 129 p, s, input := ptest.CreateList([]int{1, 2, 3}) 130 avg := globallySumInts(s, input) 131 passert.Equals(s, avg, int(6)) 132 ptest.RunAndValidate(t, p) 133 } 134 135 func TestCombine_sum_bounded(t *testing.T) { 136 p, s, input := ptest.CreateList([]int{1, 2, 3}) 137 bound := int(4) 138 avg := globallyBoundedSumInts(s, bound, input) 139 passert.Equals(s, avg, bound) 140 ptest.RunAndValidate(t, p) 141 } 142 143 type player struct { 144 Name string 145 Accuracy float64 146 } 147 148 func splitPlayer(e player) (string, float64) { 149 return e.Name, e.Accuracy 150 } 151 152 func mergePlayer(k string, v float64) player { 153 return player{Name: k, Accuracy: v} 154 } 155 156 func init() { 157 beam.RegisterFunction(splitPlayer) 158 beam.RegisterFunction(mergePlayer) 159 } 160 161 func TestCombinePerKey(t *testing.T) { 162 p, s, input := ptest.CreateList([]player{{"fred", 0.2}, {"velma", 0.4}, {"fred", 0.5}, {"velma", 1.0}, {"shaggy", 0.1}}) 163 kvs := beam.ParDo(s, splitPlayer, input) 164 avg := perKeyAverage(s, kvs) 165 results := beam.ParDo(s, mergePlayer, avg) 166 passert.Equals(s, results, player{"fred", 0.35}, player{"velma", 0.7}, player{"shaggy", 0.1}) 167 ptest.RunAndValidate(t, p) 168 } 169 170 func TestFlatten(t *testing.T) { 171 p, s := beam.NewPipelineWithRoot() 172 a := beam.CreateList(s, []int{1, 2, 3}) 173 b := beam.CreateList(s, []int{5, 7, 9}) 174 c := beam.CreateList(s, []int{4, 6, 8}) 175 merged := applyFlatten(s, a, b, c) 176 passert.Equals(s, merged, 1, 2, 3, 4, 5, 6, 7, 8, 9) 177 ptest.RunAndValidate(t, p) 178 } 179 180 func TestPartition(t *testing.T) { 181 p, s, input := ptest.CreateList([]Student{{42}, {57}, {23}, {89}, {99}, {5}}) 182 avg := applyPartition(s, input) 183 passert.Equals(s, avg, Student{42}) 184 ptest.RunAndValidate(t, p) 185 } 186 187 func TestMultipleOutputs(t *testing.T) { 188 p, s, words := ptest.CreateList([]string{"a", "the", "pjamas", "art", "candy", "MARKERmarked"}) 189 below, above, marked, lengths, mixedMarked := applyMultipleOut(s, words) 190 191 passert.Equals(s, below, "a", "the", "art") 192 passert.Equals(s, above, "pjamas", "candy", "MARKERmarked") 193 passert.Equals(s, marked, "MARKERmarked") 194 passert.Equals(s, lengths, 1, 3, 6, 3, 5, 12) 195 passert.Equals(s, mixedMarked, "MARKERmarked") 196 197 ptest.RunAndValidate(t, p) 198 } 199 200 func TestSideInputs(t *testing.T) { 201 p, s, words := ptest.CreateList([]string{"a", "the", "pjamas", "art", "candy", "garbage"}) 202 above, below := addSideInput(s, words) 203 passert.Equals(s, above, "pjamas", "candy", "garbage") 204 passert.Equals(s, below, "a", "the", "art") 205 ptest.RunAndValidate(t, p) 206 } 207 208 func TestComposite(t *testing.T) { 209 p, s, lines := ptest.CreateList([]string{ 210 "this test dataset has the word test", 211 "at least twice, because to test the Composite", 212 "CountWords, one needs test data to run it with", 213 }) 214 // [START countwords_composite_call] 215 // A Composite PTransform function is called like any other function. 216 wordCounts := CountWords(s, lines) // returns a PCollection<KV<string,int>> 217 // [END countwords_composite_call] 218 testCount := beam.ParDo(s, func(k string, v int, emit func(int)) { 219 if k == "test" { 220 emit(v) 221 } 222 }, wordCounts) 223 passert.Equals(s, testCount, 4) 224 ptest.RunAndValidate(t, p) 225 }