-
Notifications
You must be signed in to change notification settings - Fork 4.3k
/
pardo.go
83 lines (69 loc) · 2.36 KB
/
pardo.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package primitives
import (
"github.com/apache/beam/sdks/go/pkg/beam"
"github.com/apache/beam/sdks/go/pkg/beam/testing/passert"
)
func emit3Fn(elm int, emit, emit2, emit3 func(int)) {
emit(elm + 1)
emit2(elm + 2)
emit3(elm + 3)
}
// ParDoMultiOutput test a DoFn with multiple output.
func ParDoMultiOutput() *beam.Pipeline {
p, s := beam.NewPipelineWithRoot()
in := beam.Create(s, 1)
emit1, emit2, emit3 := beam.ParDo3(s, emit3Fn, in)
passert.Sum(s, emit1, "emit1", 1, 2)
passert.Sum(s, emit2, "emit2", 1, 3)
passert.Sum(s, emit3, "emit3", 1, 4)
return p
}
func sumValuesFn(_ []byte, values func(*int) bool) int {
sum := 0
var i int
for values(&i) {
sum += i
}
return sum
}
// ParDoSideInput computes the sum of ints using a side input.
func ParDoSideInput() *beam.Pipeline {
p, s := beam.NewPipelineWithRoot()
in := beam.Create(s, 1, 2, 3, 4, 5, 6, 7, 8, 9)
sub := s.Scope("subscope") // Ensure scoping works with side inputs. See: BEAM-5354
out := beam.ParDo(sub, sumValuesFn, beam.Impulse(s), beam.SideInput{Input: in})
passert.Sum(s, out, "out", 1, 45)
return p
}
func sumKVValuesFn(_ []byte, values func(*int, *int) bool) int {
sum := 0
var i, k int
for values(&i, &k) {
sum += i
sum += k
}
return sum
}
// ParDoKVSideInput computes the sum of ints using a KV side input.
func ParDoKVSideInput() *beam.Pipeline {
p, s := beam.NewPipelineWithRoot()
in := beam.Create(s, 1, 2, 3, 4, 5, 6, 7, 8, 9)
kv := beam.AddFixedKey(s, in) // i -> (0,i)
out := beam.ParDo(s, sumKVValuesFn, beam.Impulse(s), beam.SideInput{Input: kv})
passert.Sum(s, out, "out", 1, 45)
return p
}