/
translate.go
1259 lines (1125 loc) · 35.9 KB
/
translate.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
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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 graphx
import (
"context"
"fmt"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window/trigger"
v1pb "github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/graphx/v1"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/runtime/pipelinex"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/util/protox"
"github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors"
pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1"
"github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes"
)
// Model constants for interfacing with a Beam runner.
// TODO(lostluck): 2018/05/28 Extract these from their enum descriptors in the pipeline_v1 proto
const (
URNImpulse = "beam:transform:impulse:v1"
URNParDo = "beam:transform:pardo:v1"
URNFlatten = "beam:transform:flatten:v1"
URNGBK = "beam:transform:group_by_key:v1"
URNReshuffle = "beam:transform:reshuffle:v1"
URNCombinePerKey = "beam:transform:combine_per_key:v1"
URNWindow = "beam:transform:window_into:v1"
// URNIterableSideInput = "beam:side_input:iterable:v1"
URNMultimapSideInput = "beam:side_input:multimap:v1"
URNGlobalWindowsWindowFn = "beam:window_fn:global_windows:v1"
URNFixedWindowsWindowFn = "beam:window_fn:fixed_windows:v1"
URNSlidingWindowsWindowFn = "beam:window_fn:sliding_windows:v1"
URNSessionsWindowFn = "beam:window_fn:session_windows:v1"
// SDK constants
URNDoFn = "beam:go:transform:dofn:v1"
URNIterableSideInputKey = "beam:go:transform:iterablesideinputkey:v1"
URNReshuffleInput = "beam:go:transform:reshuffleinput:v1"
URNReshuffleOutput = "beam:go:transform:reshuffleoutput:v1"
URNWindowMappingGlobal = "beam:go:windowmapping:global:v1"
URNWindowMappingFixed = "beam:go:windowmapping:fixed:v1"
URNWindowMappingSliding = "beam:go:windowmapping:sliding:v1"
URNLegacyProgressReporting = "beam:protocol:progress_reporting:v0"
URNMultiCore = "beam:protocol:multi_core_bundle_processing:v1"
URNRequiresSplittableDoFn = "beam:requirement:pardo:splittable_dofn:v1"
URNArtifactGoWorker = "beam:artifact:type:go_worker_binary:v1"
URNArtifactStagingTo = "beam:artifact:role:staging_to:v1"
)
func goCapabilities() []string {
capabilities := []string{
URNLegacyProgressReporting,
URNMultiCore,
// TOOD(BEAM-9614): Make this versioned.
"beam:version:sdk_base:go",
}
return append(capabilities, knownStandardCoders()...)
}
// CreateEnvironment produces the appropriate payload for the type of environment.
func CreateEnvironment(ctx context.Context, urn string, extractEnvironmentConfig func(context.Context) string) (*pipepb.Environment, error) {
var serializedPayload []byte
switch urn {
case "beam:env:process:v1":
// TODO Support process based SDK Harness.
return nil, errors.Errorf("unsupported environment %v", urn)
case "beam:env:external:v1":
config := extractEnvironmentConfig(ctx)
payload := &pipepb.ExternalPayload{Endpoint: &pipepb.ApiServiceDescriptor{Url: config}}
serializedPayload = protox.MustEncode(payload)
case "beam:env:docker:v1":
fallthrough
default:
config := extractEnvironmentConfig(ctx)
payload := &pipepb.DockerPayload{ContainerImage: config}
serializedPayload = protox.MustEncode(payload)
}
return &pipepb.Environment{
Urn: urn,
Payload: serializedPayload,
Capabilities: goCapabilities(),
Dependencies: []*pipepb.ArtifactInformation{
{
TypeUrn: URNArtifactGoWorker,
RoleUrn: URNArtifactStagingTo,
RolePayload: protox.MustEncode(&pipepb.ArtifactStagingToRolePayload{
StagedName: "worker",
}),
},
},
}, nil
}
// TODO(herohde) 11/6/2017: move some of the configuration into the graph during construction.
// Options for marshalling a graph into a model pipeline.
type Options struct {
// Environment used to run the user code.
Environment *pipepb.Environment
}
// Marshal converts a graph to a model pipeline.
func Marshal(edges []*graph.MultiEdge, opt *Options) (*pipepb.Pipeline, error) {
if len(edges) == 0 {
return nil, errors.New("empty graph")
}
tree := NewScopeTree(edges)
m := newMarshaller(opt)
for _, edge := range tree.Edges {
_, err := m.addMultiEdge(edge)
if err != nil {
return nil, err
}
}
for _, t := range tree.Children {
_, err := m.addScopeTree(t)
if err != nil {
return nil, err
}
}
p := &pipepb.Pipeline{
Components: m.build(),
Requirements: m.getRequirements(),
}
p, err := pipelinex.Normalize(p)
if err != nil {
return nil, err
}
// If there are external transforms that need expanding, do it now.
if m.needsExpansion {
// Remap outputs of expanded external transforms to be the inputs for all downstream consumers
purgeOutputInput(edges, p)
// Merge the expanded components into the existing pipeline
mergeExpandedWithPipeline(edges, p)
}
return p, nil
}
type marshaller struct {
opt *Options
transforms map[string]*pipepb.PTransform
pcollections map[string]*pipepb.PCollection
windowing map[string]*pipepb.WindowingStrategy
environments map[string]*pipepb.Environment
requirements map[string]bool
coders *CoderMarshaller
windowing2id map[string]string
needsExpansion bool // Indicates external transforms need to be expanded.
}
func newMarshaller(opt *Options) *marshaller {
return &marshaller{
opt: opt,
transforms: make(map[string]*pipepb.PTransform),
pcollections: make(map[string]*pipepb.PCollection),
windowing: make(map[string]*pipepb.WindowingStrategy),
environments: make(map[string]*pipepb.Environment),
requirements: make(map[string]bool),
coders: NewCoderMarshaller(),
windowing2id: make(map[string]string),
}
}
func (m *marshaller) build() *pipepb.Components {
return &pipepb.Components{
Transforms: m.transforms,
Pcollections: m.pcollections,
WindowingStrategies: m.windowing,
Environments: m.environments,
Coders: m.coders.Build(),
}
}
func (m *marshaller) getRequirements() []string {
var reqs []string
for req, ok := range m.requirements {
if ok {
reqs = append(reqs, req)
}
}
return reqs
}
func (m *marshaller) addScopeTree(s *ScopeTree) (string, error) {
id := scopeID(s.Scope.Scope)
if _, exists := m.transforms[id]; exists {
return id, nil
}
var subtransforms []string
for _, edge := range s.Edges {
ids, err := m.addMultiEdge(edge)
if err != nil {
return "", errors.Wrapf(err, "failed to add scope tree: %v", s)
}
subtransforms = append(subtransforms, ids...)
}
for _, tree := range s.Children {
id, err := m.addScopeTree(tree)
if err != nil {
return "", errors.Wrapf(err, "failed to add scope tree: %v", s)
}
subtransforms = append(subtransforms, id)
}
transform := &pipepb.PTransform{
UniqueName: s.Scope.Name,
Subtransforms: subtransforms,
EnvironmentId: m.addDefaultEnv(),
}
if err := m.updateIfCombineComposite(s, transform); err != nil {
return "", errors.Wrapf(err, "failed to add scope tree: %v", s)
}
m.transforms[id] = transform
return id, nil
}
// updateIfCombineComposite examines the scope tree and sets the PTransform Spec
// to be a CombinePerKey with a CombinePayload if it's a liftable composite.
// Beam Portability requires that composites contain an implementation for runners
// that don't understand the URN and Payload, which this lightly checks for.
func (m *marshaller) updateIfCombineComposite(s *ScopeTree, transform *pipepb.PTransform) error {
if s.Scope.Name != graph.CombinePerKeyScope ||
len(s.Edges) != 2 ||
len(s.Edges[0].Edge.Input) != 1 ||
len(s.Edges[1].Edge.Output) != 1 ||
s.Edges[1].Edge.Op != graph.Combine {
return nil
}
edge := s.Edges[1].Edge
acID, err := m.coders.Add(edge.AccumCoder)
if err != nil {
return errors.Wrapf(err, "failed to update PTransform spec: %v", transform)
}
mustEncodeMultiEdge, err := mustEncodeMultiEdgeBase64(edge)
if err != nil {
return errors.Wrapf(err, "failed to update PTransform spec: %v", transform)
}
payload := &pipepb.CombinePayload{
CombineFn: &pipepb.FunctionSpec{
Urn: URNDoFn,
Payload: []byte(mustEncodeMultiEdge),
},
AccumulatorCoderId: acID,
}
transform.Spec = &pipepb.FunctionSpec{Urn: URNCombinePerKey, Payload: protox.MustEncode(payload)}
return nil
}
func getSideWindowMappingUrn(winFn *window.Fn) string {
var mappingUrn string
switch winFn.Kind {
case window.GlobalWindows:
mappingUrn = URNWindowMappingGlobal
case window.FixedWindows:
mappingUrn = URNWindowMappingFixed
case window.SlidingWindows:
mappingUrn = URNWindowMappingSliding
case window.Sessions:
panic("session windowing is not supported for side inputs")
}
return mappingUrn
}
func (m *marshaller) addMultiEdge(edge NamedEdge) ([]string, error) {
handleErr := func(err error) ([]string, error) {
return nil, errors.Wrapf(err, "failed to add input kind: %v", edge)
}
id := edgeID(edge.Edge)
if _, exists := m.transforms[id]; exists {
return []string{id}, nil
}
switch {
case edge.Edge.Op == graph.CoGBK && len(edge.Edge.Input) > 1:
cogbkID, err := m.expandCoGBK(edge)
if err != nil {
return handleErr(err)
}
return []string{cogbkID}, nil
case edge.Edge.Op == graph.Reshuffle:
reshuffleID, err := m.expandReshuffle(edge)
if err != nil {
return handleErr(err)
}
return []string{reshuffleID}, nil
case edge.Edge.Op == graph.External:
if edge.Edge.External != nil {
if edge.Edge.External.Expanded != nil {
m.needsExpansion = true
}
}
if edge.Edge.Payload == nil {
edgeID, err := m.expandCrossLanguage(edge)
if err != nil {
return handleErr(err)
}
return []string{edgeID}, nil
}
}
inputs := make(map[string]string)
for i, in := range edge.Edge.Input {
if _, err := m.addNode(in.From); err != nil {
return handleErr(err)
}
inputs[fmt.Sprintf("i%v", i)] = nodeID(in.From)
}
outputs := make(map[string]string)
for i, out := range edge.Edge.Output {
if _, err := m.addNode(out.To); err != nil {
return handleErr(err)
}
outputs[fmt.Sprintf("i%v", i)] = nodeID(out.To)
}
var annotations map[string][]byte
// allPIds tracks additional PTransformIDs generated for the pipeline
var allPIds []string
var spec *pipepb.FunctionSpec
switch edge.Edge.Op {
case graph.Impulse:
spec = &pipepb.FunctionSpec{Urn: URNImpulse}
case graph.ParDo:
si := make(map[string]*pipepb.SideInput)
for i, in := range edge.Edge.Input {
switch in.Kind {
case graph.Main:
// ignore: not a side input
case graph.Singleton, graph.Slice, graph.Iter, graph.ReIter:
// The only supported form of side input is MultiMap, but we
// want just iteration. So we must manually add a fixed key,
// "", even if the input is already KV.
out := fmt.Sprintf("%v_keyed%v_%v", nodeID(in.From), edgeID(edge.Edge), i)
coderId, err := m.coders.Add(makeBytesKeyedCoder(in.From.Coder))
if err != nil {
return handleErr(err)
}
if _, err := m.makeNode(out, coderId, in.From); err != nil {
return handleErr(err)
}
payload := &pipepb.ParDoPayload{
DoFn: &pipepb.FunctionSpec{
Urn: URNIterableSideInputKey,
Payload: []byte(protox.MustEncodeBase64(&v1pb.TransformPayload{
Urn: URNIterableSideInputKey,
})),
},
}
keyedID := fmt.Sprintf("%v_keyed%v", edgeID(edge.Edge), i)
keyed := &pipepb.PTransform{
UniqueName: keyedID,
Spec: &pipepb.FunctionSpec{
Urn: URNParDo,
Payload: protox.MustEncode(payload),
},
Inputs: map[string]string{"i0": nodeID(in.From)},
Outputs: map[string]string{"i0": out},
EnvironmentId: m.addDefaultEnv(),
}
m.transforms[keyedID] = keyed
allPIds = append(allPIds, keyedID)
// Fixup input map
inputs[fmt.Sprintf("i%v", i)] = out
siWfn := in.From.WindowingStrategy().Fn
mappingUrn := getSideWindowMappingUrn(siWfn)
siWSpec, err := makeWindowFn(siWfn)
if err != nil {
return nil, err
}
si[fmt.Sprintf("i%v", i)] = &pipepb.SideInput{
AccessPattern: &pipepb.FunctionSpec{
Urn: URNMultimapSideInput,
},
ViewFn: &pipepb.FunctionSpec{
Urn: "foo",
},
WindowMappingFn: &pipepb.FunctionSpec{
Urn: mappingUrn,
Payload: siWSpec.Payload,
},
}
case graph.Map, graph.MultiMap:
// Already in a MultiMap form, don't need to add a fixed key.
// Get window mapping, arrange proto field.
siWfn := in.From.WindowingStrategy().Fn
mappingUrn := getSideWindowMappingUrn(siWfn)
siWSpec, err := makeWindowFn(siWfn)
if err != nil {
return nil, err
}
si[fmt.Sprintf("i%v", i)] = &pipepb.SideInput{
AccessPattern: &pipepb.FunctionSpec{
Urn: URNMultimapSideInput,
},
ViewFn: &pipepb.FunctionSpec{
Urn: "foo",
},
WindowMappingFn: &pipepb.FunctionSpec{
Urn: mappingUrn,
Payload: siWSpec.Payload,
},
}
default:
return nil, errors.Errorf("unexpected input kind: %v", edge)
}
}
mustEncodeMultiEdge, err := mustEncodeMultiEdgeBase64(edge.Edge)
if err != nil {
return handleErr(err)
}
payload := &pipepb.ParDoPayload{
DoFn: &pipepb.FunctionSpec{
Urn: URNDoFn,
Payload: []byte(mustEncodeMultiEdge),
},
SideInputs: si,
}
if edge.Edge.DoFn.IsSplittable() {
coderId, err := m.coders.Add(edge.Edge.RestrictionCoder)
if err != nil {
return handleErr(err)
}
payload.RestrictionCoderId = coderId
m.requirements[URNRequiresSplittableDoFn] = true
}
spec = &pipepb.FunctionSpec{Urn: URNParDo, Payload: protox.MustEncode(payload)}
annotations = edge.Edge.DoFn.Annotations()
case graph.Combine:
mustEncodeMultiEdge, err := mustEncodeMultiEdgeBase64(edge.Edge)
if err != nil {
return handleErr(err)
}
payload := &pipepb.ParDoPayload{
DoFn: &pipepb.FunctionSpec{
Urn: URNDoFn,
Payload: []byte(mustEncodeMultiEdge),
},
}
spec = &pipepb.FunctionSpec{Urn: URNParDo, Payload: protox.MustEncode(payload)}
case graph.Flatten:
spec = &pipepb.FunctionSpec{Urn: URNFlatten}
case graph.CoGBK:
spec = &pipepb.FunctionSpec{Urn: URNGBK}
case graph.WindowInto:
windowFn, err := makeWindowFn(edge.Edge.WindowFn)
if err != nil {
return handleErr(err)
}
payload := &pipepb.WindowIntoPayload{
WindowFn: windowFn,
}
spec = &pipepb.FunctionSpec{Urn: URNWindow, Payload: protox.MustEncode(payload)}
case graph.External:
pyld := edge.Edge.Payload
spec = &pipepb.FunctionSpec{Urn: pyld.URN, Payload: pyld.Data}
if len(pyld.InputsMap) != 0 {
if got, want := len(pyld.InputsMap), len(edge.Edge.Input); got != want {
return handleErr(errors.Errorf("mismatch'd counts between External tags (%v) and inputs (%v)", got, want))
}
inputs = make(map[string]string)
for tag, in := range InboundTagToNode(pyld.InputsMap, edge.Edge.Input) {
if _, err := m.addNode(in); err != nil {
return handleErr(err)
}
inputs[tag] = nodeID(in)
}
}
if len(pyld.OutputsMap) != 0 {
if got, want := len(pyld.OutputsMap), len(edge.Edge.Output); got != want {
return handleErr(errors.Errorf("mismatch'd counts between External tags (%v) and outputs (%v)", got, want))
}
outputs = make(map[string]string)
for tag, out := range OutboundTagToNode(pyld.OutputsMap, edge.Edge.Output) {
if _, err := m.addNode(out); err != nil {
return handleErr(err)
}
outputs[tag] = nodeID(out)
}
}
default:
err := errors.Errorf("unexpected opcode: %v", edge.Edge.Op)
return handleErr(err)
}
var transformEnvID = ""
if !(spec.Urn == URNGBK || spec.Urn == URNImpulse) {
transformEnvID = m.addDefaultEnv()
}
transform := &pipepb.PTransform{
UniqueName: edge.Name,
Spec: spec,
Inputs: inputs,
Outputs: outputs,
EnvironmentId: transformEnvID,
Annotations: annotations,
}
m.transforms[id] = transform
allPIds = append(allPIds, id)
return allPIds, nil
}
func (m *marshaller) expandCrossLanguage(namedEdge NamedEdge) (string, error) {
edge := namedEdge.Edge
id := edgeID(edge)
inputs := make(map[string]string)
for tag, n := range ExternalInputs(edge) {
if _, err := m.addNode(n); err != nil {
return "", errors.Wrapf(err, "failed to expand cross language transform for edge: %v", namedEdge)
}
// Ignore tag if it is a dummy UnnamedInputTag
if tag == graph.UnnamedInputTag {
tag = fmt.Sprintf("i%v", edge.External.InputsMap[tag])
}
inputs[tag] = nodeID(n)
}
spec := &pipepb.FunctionSpec{
Urn: edge.External.Urn,
Payload: edge.External.Payload,
}
transform := &pipepb.PTransform{
UniqueName: namedEdge.Name,
Spec: spec,
Inputs: inputs,
EnvironmentId: m.addDefaultEnv(),
}
if edge.External.Expanded != nil {
// Outputs need to temporarily match format of unnamed Go SDK Nodes.
// After the initial pipeline is constructed, these will be used to correctly
// map consumers of these outputs to the expanded transform's outputs.
outputs := make(map[string]string)
for i, out := range edge.Output {
if _, err := m.addNode(out.To); err != nil {
return "", errors.Wrapf(err, "failed to expand cross language transform for edge: %v", namedEdge)
}
outputs[fmt.Sprintf("i%v", i)] = nodeID(out.To)
}
transform.Outputs = outputs
environment, err := ExpandedTransform(edge.External.Expanded)
if err != nil {
return "", errors.Wrapf(err, "failed to expand cross language transform for edge: %v", namedEdge)
}
transform.EnvironmentId = environment.EnvironmentId
}
m.transforms[id] = transform
return id, nil
}
func (m *marshaller) expandCoGBK(edge NamedEdge) (string, error) {
// TODO(BEAM-490): replace once CoGBK is a primitive. For now, we have to translate
// CoGBK with multiple PCollections as described in cogbk.go.
handleErr := func(err error) (string, error) {
return "", errors.Wrapf(err, "failed to expand CoGBK transform for edge: %v", edge)
}
id := edgeID(edge.Edge)
kvCoder, err := MakeKVUnionCoder(edge.Edge)
if err != nil {
return handleErr(err)
}
kvCoderID, err := m.coders.Add(kvCoder)
if err != nil {
return handleErr(err)
}
gbkCoder, err := MakeGBKUnionCoder(edge.Edge)
if err != nil {
return handleErr(err)
}
gbkCoderID, err := m.coders.Add(gbkCoder)
if err != nil {
return handleErr(err)
}
var subtransforms []string
inputs := make(map[string]string)
for i, in := range edge.Edge.Input {
if _, err := m.addNode(in.From); err != nil {
return handleErr(err)
}
out := fmt.Sprintf("%v_%v_inject%v", nodeID(in.From), id, i)
if _, err := m.makeNode(out, kvCoderID, in.From); err != nil {
return handleErr(err)
}
// Inject(i)
injectID := fmt.Sprintf("%v_inject%v", id, i)
payload := &pipepb.ParDoPayload{
DoFn: &pipepb.FunctionSpec{
Urn: URNInject,
Payload: []byte(protox.MustEncodeBase64(&v1pb.TransformPayload{
Urn: URNInject,
Inject: &v1pb.InjectPayload{N: (int32)(i)},
})),
},
}
inject := &pipepb.PTransform{
UniqueName: injectID,
Spec: &pipepb.FunctionSpec{
Urn: URNParDo,
Payload: protox.MustEncode(payload),
},
Inputs: map[string]string{"i0": nodeID(in.From)},
Outputs: map[string]string{"i0": out},
EnvironmentId: m.addDefaultEnv(),
}
m.transforms[injectID] = inject
subtransforms = append(subtransforms, injectID)
inputs[fmt.Sprintf("i%v", i)] = out
}
outNode := edge.Edge.Output[0].To
// Flatten
out := fmt.Sprintf("%v_flatten", nodeID(outNode))
if _, err := m.makeNode(out, kvCoderID, outNode); err != nil {
return handleErr(err)
}
flattenID := fmt.Sprintf("%v_flatten", id)
flatten := &pipepb.PTransform{
UniqueName: flattenID,
Spec: &pipepb.FunctionSpec{Urn: URNFlatten},
Inputs: inputs,
Outputs: map[string]string{"i0": out},
EnvironmentId: m.addDefaultEnv(),
}
m.transforms[flattenID] = flatten
subtransforms = append(subtransforms, flattenID)
// CoGBK
gbkOut := fmt.Sprintf("%v_out", nodeID(outNode))
if _, err := m.makeNode(gbkOut, gbkCoderID, outNode); err != nil {
return handleErr(err)
}
gbkID := fmt.Sprintf("%v_gbk", id)
gbk := &pipepb.PTransform{
UniqueName: gbkID,
Spec: &pipepb.FunctionSpec{Urn: URNGBK},
Inputs: map[string]string{"i0": out},
Outputs: map[string]string{"i0": gbkOut},
}
m.transforms[gbkID] = gbk
subtransforms = append(subtransforms, gbkID)
// Expand
if _, err := m.addNode(outNode); err != nil {
return handleErr(err)
}
expandID := fmt.Sprintf("%v_expand", id)
payload := &pipepb.ParDoPayload{
DoFn: &pipepb.FunctionSpec{
Urn: URNExpand,
Payload: []byte(protox.MustEncodeBase64(&v1pb.TransformPayload{
Urn: URNExpand,
})),
},
}
expand := &pipepb.PTransform{
UniqueName: expandID,
Spec: &pipepb.FunctionSpec{
Urn: URNParDo,
Payload: protox.MustEncode(payload),
},
Inputs: map[string]string{"i0": gbkOut},
Outputs: map[string]string{"i0": nodeID(outNode)},
EnvironmentId: m.addDefaultEnv(),
}
m.transforms[id] = expand
subtransforms = append(subtransforms, id)
// Add composite for visualization
cogbkID := fmt.Sprintf("%v_cogbk", id)
m.transforms[cogbkID] = &pipepb.PTransform{
UniqueName: edge.Name,
Subtransforms: subtransforms,
EnvironmentId: m.addDefaultEnv(),
}
return cogbkID, nil
}
// expandReshuffle translates resharding to a composite reshuffle
// transform.
//
// With proper runner support, the SDK doesn't need to do anything.
// However, we still need to provide a backup plan in terms of other
// PTransforms in the event the runner doesn't have a native implementation.
//
// In particular, the "backup plan" needs to:
//
// * Encode the windowed element, preserving timestamps.
// * Add random keys to the encoded windowed element []bytes
// * GroupByKey (in the global window).
// * Explode the resulting elements list.
// * Decode the windowed element []bytes.
//
// While a simple reshard can be written in user terms, (timestamps and windows
// are accessible to user functions) there are some framework internal
// optimizations that can be done if the framework is aware of the reshard, though
// ideally this is handled on the runner side.
//
// User code is able to write reshards, but it's easier to access
// the window coders framework side, which is critical for the reshard
// to function with unbounded inputs.
func (m *marshaller) expandReshuffle(edge NamedEdge) (string, error) {
handleErr := func(err error) (string, error) {
return "", errors.Wrapf(err, "failed to expand Reshuffle transform for edge: %v", edge)
}
id := edgeID(edge.Edge)
kvCoder, err := makeUnionCoder()
if err != nil {
return handleErr(err)
}
kvCoderID, err := m.coders.Add(kvCoder)
if err != nil {
return handleErr(err)
}
gbkCoderID, err := m.coders.Add(coder.NewCoGBK(kvCoder.Components))
if err != nil {
return handleErr(err)
}
var subtransforms []string
in := edge.Edge.Input[0]
origInput, err := m.addNode(in.From)
if err != nil {
return handleErr(err)
}
// We need to preserve the old windowing/triggering here
// for re-instatement after the GBK.
preservedWSId := m.pcollections[origInput].GetWindowingStrategyId()
// Get the windowing strategy from before:
postReify := fmt.Sprintf("%v_%v_reifyts", nodeID(in.From), id)
if _, err := m.makeNode(postReify, kvCoderID, in.From); err != nil {
return handleErr(err)
}
// We need to replace postReify's windowing strategy with one appropriate
// for reshuffles.
{
wfn := window.NewGlobalWindows()
windowFn, err := makeWindowFn(wfn)
if err != nil {
return handleErr(err)
}
coderId, err := makeWindowCoder(wfn)
if err != nil {
return handleErr(err)
}
windowCoderId, err := m.coders.AddWindowCoder(coderId)
if err != nil {
return handleErr(err)
}
m.pcollections[postReify].WindowingStrategyId =
m.internWindowingStrategy(&pipepb.WindowingStrategy{
// Not segregated by time...
WindowFn: windowFn,
// ...output after every element is received...
Trigger: &pipepb.Trigger{
Trigger: &pipepb.Trigger_Always_{
Always: &pipepb.Trigger_Always{},
},
},
// ...and after outputing, discard the output elements...
AccumulationMode: pipepb.AccumulationMode_DISCARDING,
// ...and since every pane should have 1 element,
// try to preserve the timestamp.
OutputTime: pipepb.OutputTime_EARLIEST_IN_PANE,
// Defaults copied from MarshalWindowingStrategy.
// TODO(BEAM-3304): migrate to user side operations once trigger support is in.
EnvironmentId: m.addDefaultEnv(),
MergeStatus: pipepb.MergeStatus_NON_MERGING,
WindowCoderId: windowCoderId,
ClosingBehavior: pipepb.ClosingBehavior_EMIT_IF_NONEMPTY,
AllowedLateness: 0,
OnTimeBehavior: pipepb.OnTimeBehavior_FIRE_ALWAYS,
})
}
// Inputs (i)
inputID := fmt.Sprintf("%v_reifyts", id)
payload := &pipepb.ParDoPayload{
DoFn: &pipepb.FunctionSpec{
Urn: URNReshuffleInput,
Payload: []byte(protox.MustEncodeBase64(&v1pb.TransformPayload{
Urn: URNReshuffleInput,
})),
},
}
input := &pipepb.PTransform{
UniqueName: inputID,
Spec: &pipepb.FunctionSpec{
Urn: URNParDo,
Payload: protox.MustEncode(payload),
},
Inputs: map[string]string{"i0": nodeID(in.From)},
Outputs: map[string]string{"i0": postReify},
EnvironmentId: m.addDefaultEnv(),
}
m.transforms[inputID] = input
subtransforms = append(subtransforms, inputID)
outNode := edge.Edge.Output[0].To
// GBK
gbkOut := fmt.Sprintf("%v_out", nodeID(outNode))
if _, err := m.makeNode(gbkOut, gbkCoderID, outNode); err != nil {
return handleErr(err)
}
gbkID := fmt.Sprintf("%v_gbk", id)
gbk := &pipepb.PTransform{
UniqueName: gbkID,
Spec: &pipepb.FunctionSpec{Urn: URNGBK},
Inputs: map[string]string{"i0": postReify},
Outputs: map[string]string{"i0": gbkOut},
}
m.transforms[gbkID] = gbk
subtransforms = append(subtransforms, gbkID)
// Expand
outPCol, err := m.addNode(outNode)
if err != nil {
return handleErr(err)
}
m.pcollections[outPCol].WindowingStrategyId = preservedWSId
outputID := fmt.Sprintf("%v_unreify", id)
outputPayload := &pipepb.ParDoPayload{
DoFn: &pipepb.FunctionSpec{
Urn: URNReshuffleOutput,
Payload: []byte(protox.MustEncodeBase64(&v1pb.TransformPayload{
Urn: URNReshuffleOutput,
})),
},
}
output := &pipepb.PTransform{
UniqueName: outputID,
Spec: &pipepb.FunctionSpec{
Urn: URNParDo,
Payload: protox.MustEncode(outputPayload),
},
Inputs: map[string]string{"i0": gbkOut},
Outputs: map[string]string{"i0": nodeID(outNode)},
EnvironmentId: m.addDefaultEnv(),
}
m.transforms[id] = output
subtransforms = append(subtransforms, id)
// Add composite for visualization, or runner optimization
reshuffleID := fmt.Sprintf("%v_reshuffle", id)
m.transforms[reshuffleID] = &pipepb.PTransform{
UniqueName: edge.Name,
Subtransforms: subtransforms,
Spec: &pipepb.FunctionSpec{
Urn: URNReshuffle,
},
EnvironmentId: m.addDefaultEnv(),
}
return reshuffleID, nil
}
func (m *marshaller) addNode(n *graph.Node) (string, error) {
id := nodeID(n)
if _, exists := m.pcollections[id]; exists {
return id, nil
}
// TODO(herohde) 11/15/2017: expose UniqueName to user.
cid, err := m.coders.Add(n.Coder)
if err != nil {
return "", err
}
return m.makeNode(id, cid, n)
}
func (m *marshaller) makeNode(id, cid string, n *graph.Node) (string, error) {
windowingStrategyId, err := m.addWindowingStrategy(n.WindowingStrategy())
if err != nil {
return "", errors.Wrapf(err, "failed to make node %v with node id %v", n, id)
}
col := &pipepb.PCollection{
UniqueName: id,
CoderId: cid,
IsBounded: boolToBounded(n.Bounded()),
WindowingStrategyId: windowingStrategyId,
}
m.pcollections[id] = col
return id, nil
}
func boolToBounded(bounded bool) pipepb.IsBounded_Enum {
if bounded {
return pipepb.IsBounded_BOUNDED
}
return pipepb.IsBounded_UNBOUNDED
}
const defaultEnvId = "go"
func (m *marshaller) addDefaultEnv() string {
if _, exists := m.environments[defaultEnvId]; !exists {
m.environments[defaultEnvId] = m.opt.Environment
}
return defaultEnvId
}
func (m *marshaller) addWindowingStrategy(w *window.WindowingStrategy) (string, error) {
ws, err := MarshalWindowingStrategy(m.coders, w)
if err != nil {
return "", errors.Wrapf(err, "failed to add window strategy %v", w)
}
ws.EnvironmentId = m.addDefaultEnv()
return m.internWindowingStrategy(ws), nil
}
func (m *marshaller) internWindowingStrategy(w *pipepb.WindowingStrategy) string {