-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
worker.go
627 lines (567 loc) · 19.1 KB
/
worker.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
// 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 worker handles interactions with SDK side workers, representing
// the worker services, communicating with those services, and SDK environments.
package worker
import (
"bytes"
"context"
"fmt"
"io"
"math"
"net"
"strconv"
"strings"
"sync"
"sync/atomic"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core"
"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/runtime/exec"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/typex"
fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1"
pipepb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/pipeline_v1"
"github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal/engine"
"github.com/apache/beam/sdks/v2/go/pkg/beam/runners/prism/internal/urns"
"golang.org/x/exp/slog"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"google.golang.org/protobuf/encoding/prototext"
"google.golang.org/protobuf/types/known/structpb"
)
// A W manages worker environments, sending them work
// that they're able to execute, and manages the server
// side handlers for FnAPI RPCs.
type W struct {
fnpb.UnimplementedBeamFnControlServer
fnpb.UnimplementedBeamFnDataServer
fnpb.UnimplementedBeamFnStateServer
fnpb.UnimplementedBeamFnLoggingServer
fnpb.UnimplementedProvisionServiceServer
ID, Env string
JobKey, ArtifactEndpoint string
EnvPb *pipepb.Environment
PipelineOptions *structpb.Struct
// Server management
lis net.Listener
server *grpc.Server
// These are the ID sources
inst uint64
connected, stopped atomic.Bool
InstReqs chan *fnpb.InstructionRequest
DataReqs chan *fnpb.Elements
mu sync.Mutex
activeInstructions map[string]controlResponder // Active instructions keyed by InstructionID
Descriptors map[string]*fnpb.ProcessBundleDescriptor // Stages keyed by PBDID
}
type controlResponder interface {
Respond(*fnpb.InstructionResponse)
}
// New starts the worker server components of FnAPI Execution.
func New(id, env string) *W {
lis, err := net.Listen("tcp", ":0")
if err != nil {
panic(fmt.Sprintf("failed to listen: %v", err))
}
opts := []grpc.ServerOption{
grpc.MaxRecvMsgSize(math.MaxInt32),
}
wk := &W{
ID: id,
Env: env,
lis: lis,
server: grpc.NewServer(opts...),
InstReqs: make(chan *fnpb.InstructionRequest, 10),
DataReqs: make(chan *fnpb.Elements, 10),
activeInstructions: make(map[string]controlResponder),
Descriptors: make(map[string]*fnpb.ProcessBundleDescriptor),
}
slog.Debug("Serving Worker components", slog.String("endpoint", wk.Endpoint()))
fnpb.RegisterBeamFnControlServer(wk.server, wk)
fnpb.RegisterBeamFnDataServer(wk.server, wk)
fnpb.RegisterBeamFnLoggingServer(wk.server, wk)
fnpb.RegisterBeamFnStateServer(wk.server, wk)
fnpb.RegisterProvisionServiceServer(wk.server, wk)
return wk
}
func (wk *W) Endpoint() string {
_, port, _ := net.SplitHostPort(wk.lis.Addr().String())
return fmt.Sprintf("localhost:%v", port)
}
// Serve serves on the started listener. Blocks.
func (wk *W) Serve() {
wk.server.Serve(wk.lis)
}
func (wk *W) String() string {
return "worker[" + wk.ID + "]"
}
func (wk *W) LogValue() slog.Value {
return slog.GroupValue(
slog.String("ID", wk.ID),
slog.String("endpoint", wk.Endpoint()),
)
}
// Stop the GRPC server.
func (wk *W) Stop() {
slog.Debug("stopping", "worker", wk)
wk.stopped.Store(true)
close(wk.InstReqs)
close(wk.DataReqs)
wk.server.Stop()
wk.lis.Close()
slog.Debug("stopped", "worker", wk)
}
func (wk *W) NextInst() string {
return fmt.Sprintf("inst-%v-%03d", wk.Env, atomic.AddUint64(&wk.inst, 1))
}
// TODO set logging level.
var minsev = fnpb.LogEntry_Severity_DEBUG
func (wk *W) GetProvisionInfo(_ context.Context, _ *fnpb.GetProvisionInfoRequest) (*fnpb.GetProvisionInfoResponse, error) {
endpoint := &pipepb.ApiServiceDescriptor{
Url: wk.Endpoint(),
}
resp := &fnpb.GetProvisionInfoResponse{
Info: &fnpb.ProvisionInfo{
// TODO: Include runner capabilities with the per job configuration.
RunnerCapabilities: []string{
urns.CapabilityMonitoringInfoShortIDs,
},
LoggingEndpoint: endpoint,
ControlEndpoint: endpoint,
ArtifactEndpoint: &pipepb.ApiServiceDescriptor{
Url: wk.ArtifactEndpoint,
},
RetrievalToken: wk.JobKey,
Dependencies: wk.EnvPb.GetDependencies(),
PipelineOptions: wk.PipelineOptions,
Metadata: map[string]string{
"runner": "prism",
"runner_version": core.SdkVersion,
"variant": "test",
},
},
}
return resp, nil
}
// Logging relates SDK worker messages back to the job that spawned them.
// Messages are received from the SDK,
func (wk *W) Logging(stream fnpb.BeamFnLogging_LoggingServer) error {
for {
in, err := stream.Recv()
if err == io.EOF {
return nil
}
if err != nil {
switch status.Code(err) {
case codes.Canceled:
return nil
default:
slog.Error("logging.Recv", err, "worker", wk)
return err
}
}
for _, l := range in.GetLogEntries() {
if l.Severity >= minsev {
// TODO: Connect to the associated Job for this worker instead of
// logging locally for SDK side logging.
file := l.GetLogLocation()
i := strings.LastIndex(file, ":")
line, _ := strconv.Atoi(file[i+1:])
if i > 0 {
file = file[:i]
}
slog.LogAttrs(stream.Context(), toSlogSev(l.GetSeverity()), l.GetMessage(),
slog.Any(slog.SourceKey, &slog.Source{
File: file,
Line: line,
}),
slog.Time(slog.TimeKey, l.GetTimestamp().AsTime()),
slog.Any("worker", wk),
)
}
}
}
}
func toSlogSev(sev fnpb.LogEntry_Severity_Enum) slog.Level {
switch sev {
case fnpb.LogEntry_Severity_TRACE:
return slog.Level(-8)
case fnpb.LogEntry_Severity_DEBUG:
return slog.LevelDebug // -4
case fnpb.LogEntry_Severity_INFO:
return slog.LevelInfo // 0
case fnpb.LogEntry_Severity_NOTICE:
return slog.Level(2)
case fnpb.LogEntry_Severity_WARN:
return slog.LevelWarn // 4
case fnpb.LogEntry_Severity_ERROR:
return slog.LevelError // 8
case fnpb.LogEntry_Severity_CRITICAL:
return slog.Level(10)
}
return slog.LevelInfo
}
func (wk *W) GetProcessBundleDescriptor(ctx context.Context, req *fnpb.GetProcessBundleDescriptorRequest) (*fnpb.ProcessBundleDescriptor, error) {
desc, ok := wk.Descriptors[req.GetProcessBundleDescriptorId()]
if !ok {
return nil, fmt.Errorf("descriptor %v not found", req.GetProcessBundleDescriptorId())
}
return desc, nil
}
// Connected indicates whether the worker has connected to the control RPC.
func (wk *W) Connected() bool {
return wk.connected.Load()
}
// Stopped indicates that the worker has stopped.
func (wk *W) Stopped() bool {
return wk.stopped.Load()
}
// Control relays instructions to SDKs and back again, coordinated via unique instructionIDs.
//
// Requests come from the runner, and are sent to the client in the SDK.
func (wk *W) Control(ctrl fnpb.BeamFnControl_ControlServer) error {
wk.connected.Store(true)
done := make(chan error, 1)
go func() {
for {
resp, err := ctrl.Recv()
if err == io.EOF {
slog.Debug("ctrl.Recv finished; marking done", "worker", wk)
done <- nil // means stream is finished
return
}
if err != nil {
switch status.Code(err) {
case codes.Canceled:
done <- err // means stream is finished
return
default:
slog.Error("ctrl.Recv failed", "error", err, "worker", wk)
panic(err)
}
}
wk.mu.Lock()
if b, ok := wk.activeInstructions[resp.GetInstructionId()]; ok {
b.Respond(resp)
} else {
slog.Debug("ctrl.Recv: %v", resp)
}
wk.mu.Unlock()
}
}()
for {
select {
case req, ok := <-wk.InstReqs:
if !ok {
slog.Debug("Worker shutting down.", "worker", wk)
return nil
}
if err := ctrl.Send(req); err != nil {
return err
}
case <-ctrl.Context().Done():
wk.mu.Lock()
// Fail extant instructions
slog.Debug("SDK Disconnected", "worker", wk, "ctx_error", ctrl.Context().Err(), "outstanding_instructions", len(wk.activeInstructions))
msg := fmt.Sprintf("SDK worker disconnected: %v, %v active instructions", wk.String(), len(wk.activeInstructions))
for instID, b := range wk.activeInstructions {
b.Respond(&fnpb.InstructionResponse{
InstructionId: instID,
Error: msg,
})
}
wk.mu.Unlock()
return context.Cause(ctrl.Context())
case err := <-done:
if err != nil {
slog.Warn("Control done", "error", err, "worker", wk)
} else {
slog.Debug("Control done", "worker", wk)
}
return err
}
}
}
// Data relays elements and timer bytes to SDKs and back again, coordinated via
// ProcessBundle instructionIDs, and receiving input transforms.
//
// Data is multiplexed on a single stream for all active bundles on a worker.
func (wk *W) Data(data fnpb.BeamFnData_DataServer) error {
go func() {
for {
resp, err := data.Recv()
if err == io.EOF {
return
}
if err != nil {
switch status.Code(err) {
case codes.Canceled:
return
default:
slog.Error("data.Recv failed", err, "worker", wk)
panic(err)
}
}
wk.mu.Lock()
for _, d := range resp.GetData() {
cr, ok := wk.activeInstructions[d.GetInstructionId()]
if !ok {
slog.Info("data.Recv for unknown bundle", "response", resp)
continue
}
// Received data is always for an active ProcessBundle instruction
b := cr.(*B)
colID := b.SinkToPCollection[d.GetTransformId()]
// There might not be data, eg. for side inputs, so we need to reconcile this elsewhere for
// downstream side inputs.
if len(d.GetData()) > 0 {
b.OutputData.WriteData(colID, d.GetData())
}
if d.GetIsLast() {
b.DataOrTimerDone()
}
}
for _, t := range resp.GetTimers() {
cr, ok := wk.activeInstructions[t.GetInstructionId()]
if !ok {
slog.Info("data.Recv for unknown bundle", "response", resp)
continue
}
// Received data is always for an active ProcessBundle instruction
b := cr.(*B)
if len(t.GetTimers()) > 0 {
b.OutputData.WriteTimers(t.GetTransformId(), t.GetTimerFamilyId(), t.GetTimers())
}
if t.GetIsLast() {
b.DataOrTimerDone()
}
}
wk.mu.Unlock()
}
}()
for {
select {
case req, ok := <-wk.DataReqs:
if !ok {
return nil
}
if err := data.Send(req); err != nil {
slog.LogAttrs(context.TODO(), slog.LevelDebug, "data.Send error", slog.Any("error", err))
}
case <-data.Context().Done():
slog.Debug("Data context canceled")
return context.Cause(data.Context())
}
}
}
// State relays elements and timer bytes to SDKs and back again, coordinated via
// ProcessBundle instructionIDs, and receiving input transforms.
//
// State requests come from SDKs, and the runner responds.
func (wk *W) State(state fnpb.BeamFnState_StateServer) error {
responses := make(chan *fnpb.StateResponse)
go func() {
// This go routine creates all responses to state requests from the worker
// so we want to close the State handler when it's all done.
defer close(responses)
for {
req, err := state.Recv()
if err == io.EOF {
return
}
if err != nil {
switch status.Code(err) {
case codes.Canceled:
return
default:
slog.Error("state.Recv failed", err, "worker", wk)
panic(err)
}
}
// State requests are always for an active ProcessBundle instruction
wk.mu.Lock()
b, ok := wk.activeInstructions[req.GetInstructionId()].(*B)
wk.mu.Unlock()
if !ok {
slog.Warn("state request after bundle inactive", "instruction", req.GetInstructionId(), "worker", wk)
continue
}
switch req.GetRequest().(type) {
case *fnpb.StateRequest_Get:
// TODO: move data handling to be pcollection based.
key := req.GetStateKey()
slog.Debug("StateRequest_Get", prototext.Format(req), "bundle", b)
var data [][]byte
switch key.GetType().(type) {
case *fnpb.StateKey_IterableSideInput_:
ikey := key.GetIterableSideInput()
wKey := ikey.GetWindow()
var w typex.Window
if len(wKey) == 0 {
w = window.GlobalWindow{}
} else {
w, err = exec.MakeWindowDecoder(coder.NewIntervalWindow()).DecodeSingle(bytes.NewBuffer(wKey))
if err != nil {
panic(fmt.Sprintf("error decoding iterable side input window key %v: %v", wKey, err))
}
}
winMap := b.IterableSideInputData[SideInputKey{TransformID: ikey.GetTransformId(), Local: ikey.GetSideInputId()}]
var wins []typex.Window
for w := range winMap {
wins = append(wins, w)
}
slog.Debug(fmt.Sprintf("side input[%v][%v] I Key: %v Windows: %v", req.GetId(), req.GetInstructionId(), w, wins))
data = winMap[w]
case *fnpb.StateKey_MultimapSideInput_:
mmkey := key.GetMultimapSideInput()
wKey := mmkey.GetWindow()
var w typex.Window
if len(wKey) == 0 {
w = window.GlobalWindow{}
} else {
w, err = exec.MakeWindowDecoder(coder.NewIntervalWindow()).DecodeSingle(bytes.NewBuffer(wKey))
if err != nil {
panic(fmt.Sprintf("error decoding multimap side input window key %v: %v", wKey, err))
}
}
dKey := mmkey.GetKey()
winMap := b.MultiMapSideInputData[SideInputKey{TransformID: mmkey.GetTransformId(), Local: mmkey.GetSideInputId()}]
slog.Debug(fmt.Sprintf("side input[%v][%v] MultiMap Window: %v", req.GetId(), req.GetInstructionId(), w))
data = winMap[w][string(dKey)]
case *fnpb.StateKey_BagUserState_:
bagkey := key.GetBagUserState()
data = b.OutputData.GetBagState(engine.LinkID{Transform: bagkey.GetTransformId(), Local: bagkey.GetUserStateId()}, bagkey.GetWindow(), bagkey.GetKey())
case *fnpb.StateKey_MultimapUserState_:
mmkey := key.GetMultimapUserState()
data = b.OutputData.GetMultimapState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey(), mmkey.GetMapKey())
case *fnpb.StateKey_MultimapKeysUserState_:
mmkey := key.GetMultimapKeysUserState()
data = b.OutputData.GetMultimapKeysState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey())
default:
panic(fmt.Sprintf("unsupported StateKey Get type: %T: %v", key.GetType(), prototext.Format(key)))
}
// Encode the runner iterable (no length, just consecutive elements), and send it out.
// This is also where we can handle things like State Backed Iterables.
responses <- &fnpb.StateResponse{
Id: req.GetId(),
Response: &fnpb.StateResponse_Get{
Get: &fnpb.StateGetResponse{
Data: bytes.Join(data, []byte{}),
},
},
}
case *fnpb.StateRequest_Append:
key := req.GetStateKey()
switch key.GetType().(type) {
case *fnpb.StateKey_BagUserState_:
bagkey := key.GetBagUserState()
b.OutputData.AppendBagState(engine.LinkID{Transform: bagkey.GetTransformId(), Local: bagkey.GetUserStateId()}, bagkey.GetWindow(), bagkey.GetKey(), req.GetAppend().GetData())
case *fnpb.StateKey_MultimapUserState_:
mmkey := key.GetMultimapUserState()
b.OutputData.AppendMultimapState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey(), mmkey.GetMapKey(), req.GetAppend().GetData())
default:
panic(fmt.Sprintf("unsupported StateKey Append type: %T: %v", key.GetType(), prototext.Format(key)))
}
responses <- &fnpb.StateResponse{
Id: req.GetId(),
Response: &fnpb.StateResponse_Append{
Append: &fnpb.StateAppendResponse{},
},
}
case *fnpb.StateRequest_Clear:
key := req.GetStateKey()
switch key.GetType().(type) {
case *fnpb.StateKey_BagUserState_:
bagkey := key.GetBagUserState()
b.OutputData.ClearBagState(engine.LinkID{Transform: bagkey.GetTransformId(), Local: bagkey.GetUserStateId()}, bagkey.GetWindow(), bagkey.GetKey())
case *fnpb.StateKey_MultimapUserState_:
mmkey := key.GetMultimapUserState()
b.OutputData.ClearMultimapState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey(), mmkey.GetMapKey())
case *fnpb.StateKey_MultimapKeysUserState_:
mmkey := key.GetMultimapUserState()
b.OutputData.ClearMultimapKeysState(engine.LinkID{Transform: mmkey.GetTransformId(), Local: mmkey.GetUserStateId()}, mmkey.GetWindow(), mmkey.GetKey())
default:
panic(fmt.Sprintf("unsupported StateKey Clear type: %T: %v", key.GetType(), prototext.Format(key)))
}
responses <- &fnpb.StateResponse{
Id: req.GetId(),
Response: &fnpb.StateResponse_Clear{
Clear: &fnpb.StateClearResponse{},
},
}
default:
panic(fmt.Sprintf("unsupported StateRequest kind %T: %v", req.GetRequest(), prototext.Format(req)))
}
}
}()
for resp := range responses {
if err := state.Send(resp); err != nil {
slog.Error("state.Send error", err)
}
}
return nil
}
var chanResponderPool = sync.Pool{
New: func() any {
return &chanResponder{make(chan *fnpb.InstructionResponse, 1)}
},
}
type chanResponder struct {
Resp chan *fnpb.InstructionResponse
}
func (cr *chanResponder) Respond(resp *fnpb.InstructionResponse) {
cr.Resp <- resp
}
// sendInstruction is a helper for creating and sending worker single RPCs, blocking
// until the response returns.
func (wk *W) sendInstruction(ctx context.Context, req *fnpb.InstructionRequest) *fnpb.InstructionResponse {
cr := chanResponderPool.Get().(*chanResponder)
progInst := wk.NextInst()
wk.mu.Lock()
wk.activeInstructions[progInst] = cr
wk.mu.Unlock()
defer func() {
wk.mu.Lock()
delete(wk.activeInstructions, progInst)
wk.mu.Unlock()
chanResponderPool.Put(cr)
}()
req.InstructionId = progInst
if wk.Stopped() {
return nil
}
wk.InstReqs <- req
select {
case <-ctx.Done():
return &fnpb.InstructionResponse{
InstructionId: progInst,
Error: "context canceled before receive",
}
case resp := <-cr.Resp:
// Protos are safe as nil, so just return directly.
return resp
}
}
// MonitoringMetadata is a convenience method to request the metadata for monitoring shortIDs.
func (wk *W) MonitoringMetadata(ctx context.Context, unknownIDs []string) *fnpb.MonitoringInfosMetadataResponse {
return wk.sendInstruction(ctx, &fnpb.InstructionRequest{
Request: &fnpb.InstructionRequest_MonitoringInfos{
MonitoringInfos: &fnpb.MonitoringInfosMetadataRequest{
MonitoringInfoId: unknownIDs,
},
},
}).GetMonitoringInfos()
}