forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
/
harness.go
364 lines (309 loc) · 11.1 KB
/
harness.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
// 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 harness implements the SDK side of the Beam FnAPI.
package harness
import (
"context"
"fmt"
"io"
"sync"
"time"
"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
"github.com/apache/beam/sdks/go/pkg/beam/core/util/hooks"
"github.com/apache/beam/sdks/go/pkg/beam/internal/errors"
"github.com/apache/beam/sdks/go/pkg/beam/log"
fnpb "github.com/apache/beam/sdks/go/pkg/beam/model/fnexecution_v1"
"github.com/apache/beam/sdks/go/pkg/beam/util/grpcx"
"github.com/golang/protobuf/proto"
"google.golang.org/grpc"
)
// TODO(herohde) 2/8/2017: for now, assume we stage a full binary (not a plugin).
// Main is the main entrypoint for the Go harness. It runs at "runtime" -- not
// "pipeline-construction time" -- on each worker. It is a FnAPI client and
// ultimately responsible for correctly executing user code.
func Main(ctx context.Context, loggingEndpoint, controlEndpoint string) error {
hooks.DeserializeHooksFromOptions(ctx)
hooks.RunInitHooks(ctx)
setupRemoteLogging(ctx, loggingEndpoint)
recordHeader()
// Connect to FnAPI control server. Receive and execute work.
// TODO: setup data manager, DoFn register
conn, err := dial(ctx, controlEndpoint, 60*time.Second)
if err != nil {
return errors.Wrap(err, "failed to connect")
}
defer conn.Close()
client := fnpb.NewBeamFnControlClient(conn)
lookupDesc := func(id bundleDescriptorID) (*fnpb.ProcessBundleDescriptor, error) {
pbd, err := client.GetProcessBundleDescriptor(ctx, &fnpb.GetProcessBundleDescriptorRequest{ProcessBundleDescriptorId: string(id)})
log.Debugf(ctx, "GPBD RESP [%v]: %v, err %v", id, pbd, err)
return pbd, err
}
stub, err := client.Control(ctx)
if err != nil {
return errors.Wrapf(err, "failed to connect to control service")
}
log.Debugf(ctx, "Successfully connected to control @ %v", controlEndpoint)
// Each ProcessBundle is a sub-graph of the original one.
var wg sync.WaitGroup
respc := make(chan *fnpb.InstructionResponse, 100)
wg.Add(1)
// gRPC requires all writers to a stream be the same goroutine, so this is the
// goroutine for managing responses back to the control service.
go func() {
defer wg.Done()
for resp := range respc {
log.Debugf(ctx, "RESP: %v", proto.MarshalTextString(resp))
if err := stub.Send(resp); err != nil {
log.Errorf(ctx, "control.Send: Failed to respond: %v", err)
}
}
}()
ctrl := &control{
lookupDesc: lookupDesc,
descriptors: make(map[bundleDescriptorID]*fnpb.ProcessBundleDescriptor),
plans: make(map[bundleDescriptorID][]*exec.Plan),
active: make(map[instructionID]*exec.Plan),
failed: make(map[instructionID]error),
data: &DataChannelManager{},
state: &StateChannelManager{},
}
// gRPC requires all readers of a stream be the same goroutine, so this goroutine
// is responsible for managing the network data. All it does is pull data from
// the stream, and hand off the message to a goroutine to actually be handled,
// so as to avoid blocking the underlying network channel.
for {
req, err := stub.Recv()
if err != nil {
close(respc)
wg.Wait()
if err == io.EOF {
recordFooter()
return nil
}
return errors.Wrapf(err, "control.Recv failed")
}
// Launch a goroutine to handle the control message.
// TODO(wcn): implement a rate limiter for 'heavy' messages?
fn := func(ctx context.Context, req *fnpb.InstructionRequest) {
log.Debugf(ctx, "RECV: %v", proto.MarshalTextString(req))
recordInstructionRequest(req)
ctx = hooks.RunRequestHooks(ctx, req)
resp := ctrl.handleInstruction(ctx, req)
hooks.RunResponseHooks(ctx, req, resp)
recordInstructionResponse(resp)
if resp != nil {
respc <- resp
}
}
if req.GetProcessBundle() != nil {
// Only process bundles in a goroutine. We at least need to process instructions for
// each plan serially. Perhaps just invoke plan.Execute async?
go fn(ctx, req)
} else {
fn(ctx, req)
}
}
}
type bundleDescriptorID string
type instructionID string
type control struct {
lookupDesc func(bundleDescriptorID) (*fnpb.ProcessBundleDescriptor, error)
descriptors map[bundleDescriptorID]*fnpb.ProcessBundleDescriptor // protected by mu
// plans that are candidates for execution.
plans map[bundleDescriptorID][]*exec.Plan // protected by mu
// plans that are actively being executed.
// a plan can only be in one of these maps at any time.
active map[instructionID]*exec.Plan // protected by mu
// plans that have failed during execution
failed map[instructionID]error // protected by mu
mu sync.Mutex
data *DataChannelManager
state *StateChannelManager
}
func (c *control) getOrCreatePlan(bdID bundleDescriptorID) (*exec.Plan, error) {
c.mu.Lock()
plans, ok := c.plans[bdID]
var plan *exec.Plan
if ok && len(plans) > 0 {
plan = plans[len(plans)-1]
c.plans[bdID] = plans[:len(plans)-1]
} else {
desc, ok := c.descriptors[bdID]
if !ok {
c.mu.Unlock() // Unlock to make the lookup.
newDesc, err := c.lookupDesc(bdID)
if err != nil {
return nil, errors.Wrapf(err, "execution plan for %v not found", bdID)
}
c.mu.Lock()
c.descriptors[bdID] = newDesc
desc = newDesc
}
newPlan, err := exec.UnmarshalPlan(desc)
if err != nil {
c.mu.Unlock()
return nil, errors.Wrapf(err, "invalid bundle desc %v: %v", bdID, desc)
}
plan = newPlan
}
c.mu.Unlock()
return plan, nil
}
func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRequest) *fnpb.InstructionResponse {
instID := instructionID(req.GetInstructionId())
ctx = setInstID(ctx, instID)
switch {
case req.GetRegister() != nil:
msg := req.GetRegister()
c.mu.Lock()
for _, desc := range msg.GetProcessBundleDescriptor() {
c.descriptors[bundleDescriptorID(desc.GetId())] = desc
}
c.mu.Unlock()
return &fnpb.InstructionResponse{
InstructionId: string(instID),
Response: &fnpb.InstructionResponse_Register{
Register: &fnpb.RegisterResponse{},
},
}
case req.GetProcessBundle() != nil:
msg := req.GetProcessBundle()
// NOTE: the harness sends a 0-length process bundle request to sources (changed?)
bdID := bundleDescriptorID(msg.GetProcessBundleDescriptorId())
log.Debugf(ctx, "PB [%v]: %v", instID, msg)
plan, err := c.getOrCreatePlan(bdID)
// Make the plan active.
c.mu.Lock()
c.active[instID] = plan
c.mu.Unlock()
if err != nil {
return fail(ctx, instID, "Failed: %v", err)
}
data := NewScopedDataManager(c.data, instID)
state := NewScopedStateReader(c.state, instID)
err = plan.Execute(ctx, string(instID), exec.DataContext{Data: data, State: state})
data.Close()
state.Close()
mons, pylds := monitoring(plan)
// Move the plan back to the candidate state
c.mu.Lock()
// Mark the instruction as failed.
if err != nil {
c.failed[instID] = err
}
c.plans[bdID] = append(c.plans[bdID], plan)
delete(c.active, instID)
c.mu.Unlock()
if err != nil {
return fail(ctx, instID, "process bundle failed for instruction %v using plan %v : %v", instID, bdID, err)
}
return &fnpb.InstructionResponse{
InstructionId: string(instID),
Response: &fnpb.InstructionResponse_ProcessBundle{
ProcessBundle: &fnpb.ProcessBundleResponse{
MonitoringData: pylds,
MonitoringInfos: mons,
},
},
}
case req.GetProcessBundleProgress() != nil:
msg := req.GetProcessBundleProgress()
ref := instructionID(msg.GetInstructionId())
c.mu.Lock()
plan, ok := c.active[ref]
err := c.failed[ref]
c.mu.Unlock()
if err != nil {
return fail(ctx, instID, "failed to return progress: instruction %v failed: %v", ref, err)
}
if !ok {
return fail(ctx, instID, "failed to return progress: instruction %v not active", ref)
}
mons, pylds := monitoring(plan)
return &fnpb.InstructionResponse{
InstructionId: string(instID),
Response: &fnpb.InstructionResponse_ProcessBundleProgress{
ProcessBundleProgress: &fnpb.ProcessBundleProgressResponse{
MonitoringData: pylds,
MonitoringInfos: mons,
},
},
}
case req.GetProcessBundleSplit() != nil:
msg := req.GetProcessBundleSplit()
log.Debugf(ctx, "PB Split: %v", msg)
ref := instructionID(msg.GetInstructionId())
c.mu.Lock()
plan, ok := c.active[ref]
err := c.failed[ref]
c.mu.Unlock()
if err != nil {
return fail(ctx, instID, "failed to split: instruction %v failed: %v", ref, err)
}
if !ok {
return fail(ctx, instID, "failed to split: execution plan for %v not active", ref)
}
// Get the desired splits for the root FnAPI read operation.
ds := msg.GetDesiredSplits()[plan.SourcePTransformID()]
if ds == nil {
return fail(ctx, instID, "failed to split: desired splits for root of %v was empty.", ref)
}
split, err := plan.Split(exec.SplitPoints{Splits: ds.GetAllowedSplitPoints(), Frac: ds.GetFractionOfRemainder()})
if err != nil {
return fail(ctx, instID, "unable to split %v: %v", ref, err)
}
return &fnpb.InstructionResponse{
InstructionId: string(instID),
Response: &fnpb.InstructionResponse_ProcessBundleSplit{
ProcessBundleSplit: &fnpb.ProcessBundleSplitResponse{
ChannelSplits: []*fnpb.ProcessBundleSplitResponse_ChannelSplit{
&fnpb.ProcessBundleSplitResponse_ChannelSplit{
LastPrimaryElement: split - 1,
FirstResidualElement: split,
},
},
},
},
}
case req.GetProcessBundleProgressMetadata() != nil:
msg := req.GetProcessBundleProgressMetadata()
return &fnpb.InstructionResponse{
InstructionId: string(instID),
Response: &fnpb.InstructionResponse_ProcessBundleProgressMetadata{
ProcessBundleProgressMetadata: &fnpb.ProcessBundleProgressMetadataResponse{
MonitoringInfo: shortIdsToInfos(msg.GetMonitoringInfoId()),
},
},
}
default:
return fail(ctx, instID, "Unexpected request: %v", req)
}
}
func fail(ctx context.Context, id instructionID, format string, args ...interface{}) *fnpb.InstructionResponse {
log.Output(ctx, log.SevError, 1, fmt.Sprintf(format, args...))
dummy := &fnpb.InstructionResponse_Register{Register: &fnpb.RegisterResponse{}}
return &fnpb.InstructionResponse{
InstructionId: string(id),
Error: fmt.Sprintf(format, args...),
Response: dummy,
}
}
// dial to the specified endpoint. if timeout <=0, call blocks until
// grpc.Dial succeeds.
func dial(ctx context.Context, endpoint string, timeout time.Duration) (*grpc.ClientConn, error) {
log.Infof(ctx, "Connecting via grpc @ %s ...", endpoint)
return grpcx.Dial(ctx, endpoint, timeout)
}