forked from apache/beam
/
expand.go
125 lines (108 loc) · 4.49 KB
/
expand.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
// 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 xlangx
import (
"context"
"github.com/apache/beam/sdks/go/pkg/beam/core/graph"
"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/graphx"
"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/pipelinex"
"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
jobpb "github.com/apache/beam/sdks/go/pkg/beam/model/jobmanagement_v1"
pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
"google.golang.org/grpc"
)
// Expand expands an unexpanded graph.ExternalTransform as a
// graph.ExpandedTransform and assigns it to the ExternalTransform's Expanded
// field. This requires querying an expansion service based on the configuration
// details within the ExternalTransform.
func Expand(edge *graph.MultiEdge, ext *graph.ExternalTransform) error {
// Build the ExpansionRequest
// Obtaining the components and transform proto representing this transform
p, err := graphx.Marshal([]*graph.MultiEdge{edge}, &graphx.Options{})
if err != nil {
return errors.Wrapf(err, "unable to generate proto representation of %v", ext)
}
transforms := p.GetComponents().GetTransforms()
// Transforms consist of only External transform and composites. Composites
// should be removed from proto before submitting expansion request.
extTransformID := p.GetRootTransformIds()[0]
extTransform := transforms[extTransformID]
for extTransform.UniqueName != "External" {
delete(transforms, extTransformID)
p, err := pipelinex.Normalize(p)
if err != nil {
return err
}
extTransformID = p.GetRootTransformIds()[0]
extTransform = transforms[extTransformID]
}
// Scoping the ExternalTransform with respect to it's unique namespace, thus
// avoiding future collisions
addNamespace(extTransform, p.GetComponents(), ext.Namespace)
graphx.AddFakeImpulses(p) // Inputs need to have sources
delete(transforms, extTransformID)
// Querying the expansion service
res, err := queryExpansionService(context.Background(), p.GetComponents(), extTransform, ext.Namespace, ext.ExpansionAddr)
if err != nil {
return err
}
// Handling ExpansionResponse
// Previously added fake impulses need to be removed to avoid having
// multiple sources to the same pcollection in the graph
graphx.RemoveFakeImpulses(res.GetComponents(), res.GetTransform())
exp := &graph.ExpandedTransform{
Components: res.GetComponents(),
Transform: res.GetTransform(),
Requirements: res.GetRequirements(),
}
ext.Expanded = exp
return nil
}
// queryExpansionService submits an external transform to be expanded by the
// expansion service. The given transform should be the external transform, and
// the components are any additional components necessary for the pipeline
// snippet.
//
// Users should generally call beam.CrossLanguage to access foreign transforms
// rather than calling this function directly.
func queryExpansionService(
ctx context.Context,
comps *pipepb.Components,
transform *pipepb.PTransform,
namespace string,
expansionAddr string) (*jobpb.ExpansionResponse, error) {
// Querying Expansion Service
// Build expansion request proto.
req := &jobpb.ExpansionRequest{
Components: comps,
Transform: transform,
Namespace: namespace,
}
// Setting grpc client
conn, err := grpc.Dial(expansionAddr, grpc.WithInsecure())
if err != nil {
err = errors.Wrapf(err, "unable to connect to expansion service at %v", expansionAddr)
return nil, errors.WithContextf(err, "expanding transform with ExpansionRequest: %v", req)
}
defer conn.Close()
client := jobpb.NewExpansionServiceClient(conn)
// Handling ExpansionResponse
res, err := client.Expand(ctx, req)
if err != nil {
err = errors.Wrapf(err, "expansion failed")
return nil, errors.WithContextf(err, "expanding transform with ExpansionRequest: %v", req)
}
return res, nil
}