forked from apache/beam
/
statemgr.go
423 lines (373 loc) · 11.1 KB
/
statemgr.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
// 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
import (
"context"
"fmt"
"io"
"sync"
"sync/atomic"
"time"
"github.com/apache/beam/sdks/go/pkg/beam/core/runtime/exec"
"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/golang/protobuf/proto"
)
// ScopedStateReader scopes the global gRPC state manager to a single instruction
// for side input use. The indirection makes it easier to control access.
type ScopedStateReader struct {
mgr *StateChannelManager
instID instructionID
opened []io.Closer // track open readers to force close all
closed bool
mu sync.Mutex
}
// NewScopedStateReader returns a ScopedStateReader for the given instruction.
func NewScopedStateReader(mgr *StateChannelManager, instID instructionID) *ScopedStateReader {
return &ScopedStateReader{mgr: mgr, instID: instID}
}
// OpenSideInput opens a byte stream for reading iterable side input.
func (s *ScopedStateReader) OpenSideInput(ctx context.Context, id exec.StreamID, sideInputID string, key, w []byte) (io.ReadCloser, error) {
return s.openReader(ctx, id, func(ch *StateChannel) *stateKeyReader {
return newSideInputReader(ch, id, sideInputID, s.instID, key, w)
})
}
// OpenIterable opens a byte stream for reading unwindowed iterables from the runner.
func (s *ScopedStateReader) OpenIterable(ctx context.Context, id exec.StreamID, key []byte) (io.ReadCloser, error) {
return s.openReader(ctx, id, func(ch *StateChannel) *stateKeyReader {
return newRunnerReader(ch, s.instID, key)
})
}
func (s *ScopedStateReader) openReader(ctx context.Context, id exec.StreamID, readerFn func(*StateChannel) *stateKeyReader) (*stateKeyReader, error) {
ch, err := s.open(ctx, id.Port)
if err != nil {
return nil, err
}
s.mu.Lock()
if s.closed {
s.mu.Unlock()
return nil, errors.Errorf("instruction %v no longer processing", s.instID)
}
ret := readerFn(ch)
s.opened = append(s.opened, ret)
s.mu.Unlock()
return ret, nil
}
func (s *ScopedStateReader) open(ctx context.Context, port exec.Port) (*StateChannel, error) {
s.mu.Lock()
if s.closed {
s.mu.Unlock()
return nil, errors.Errorf("instruction %v no longer processing", s.instID)
}
localMgr := s.mgr
s.mu.Unlock()
return localMgr.Open(ctx, port) // don't hold lock over potentially slow operation
}
// Close closes all open readers.
func (s *ScopedStateReader) Close() error {
s.mu.Lock()
s.closed = true
s.mgr = nil
for _, r := range s.opened {
r.Close() // force close all opened readers
}
s.opened = nil
s.mu.Unlock()
return nil
}
type stateKeyReader struct {
instID instructionID
key *fnpb.StateKey
token []byte
buf []byte
eof bool
ch *StateChannel
closed bool
mu sync.Mutex
}
func newSideInputReader(ch *StateChannel, id exec.StreamID, sideInputID string, instID instructionID, k, w []byte) *stateKeyReader {
key := &fnpb.StateKey{
Type: &fnpb.StateKey_MultimapSideInput_{
MultimapSideInput: &fnpb.StateKey_MultimapSideInput{
TransformId: id.PtransformID,
SideInputId: sideInputID,
Window: w,
Key: k,
},
},
}
return &stateKeyReader{
instID: instID,
key: key,
ch: ch,
}
}
func newRunnerReader(ch *StateChannel, instID instructionID, k []byte) *stateKeyReader {
key := &fnpb.StateKey{
Type: &fnpb.StateKey_Runner_{
Runner: &fnpb.StateKey_Runner{
Key: k,
},
},
}
return &stateKeyReader{
instID: instID,
key: key,
ch: ch,
}
}
func (r *stateKeyReader) Read(buf []byte) (int, error) {
if r.buf == nil {
if r.eof {
return 0, io.EOF
}
// Buffer empty. Get next segment.
r.mu.Lock()
if r.closed {
r.mu.Unlock()
return 0, errors.New("side input closed")
}
localChannel := r.ch
r.mu.Unlock()
req := &fnpb.StateRequest{
// Id: set by StateChannel
InstructionId: string(r.instID),
StateKey: r.key,
Request: &fnpb.StateRequest_Get{
Get: &fnpb.StateGetRequest{
ContinuationToken: r.token,
},
},
}
resp, err := localChannel.Send(req)
if err != nil {
return 0, err
}
get := resp.GetGet()
if get == nil { // no data associated with this segment.
r.eof = true
return 0, io.EOF
}
r.token = get.GetContinuationToken()
r.buf = get.GetData()
if r.token == nil {
r.eof = true // no token == this is the last segment.
}
}
n := copy(buf, r.buf)
switch {
case n == 0 && len(buf) != 0 && r.eof:
// If no data was copied, and this is the last segment anyway, return EOF now.
// This prevent spurious zero elements.
r.buf = nil
return 0, io.EOF
case len(r.buf) == n:
r.buf = nil
default:
r.buf = r.buf[n:]
}
return n, nil
}
func (r *stateKeyReader) Close() error {
r.mu.Lock()
r.closed = true
r.ch = nil // StateChannels might be re-used if they're ok, so don't close them here.
r.mu.Unlock()
return nil
}
// StateChannelManager manages data channels over the State API. A fixed number of channels
// are generally used, each managing multiple logical byte streams. Thread-safe.
type StateChannelManager struct {
ports map[string]*StateChannel
mu sync.Mutex
}
// Open opens a R/W StateChannel over the given port.
func (m *StateChannelManager) Open(ctx context.Context, port exec.Port) (*StateChannel, error) {
m.mu.Lock()
defer m.mu.Unlock()
if m.ports == nil {
m.ports = make(map[string]*StateChannel)
}
if con, ok := m.ports[port.URL]; ok {
return con, nil
}
ch, err := newStateChannel(ctx, port)
if err != nil {
return nil, err
}
ch.forceRecreate = func(id string, err error) {
log.Warnf(ctx, "forcing StateChannel[%v] reconnection on port %v due to %v", id, port, err)
m.mu.Lock()
delete(m.ports, port.URL)
m.mu.Unlock()
}
m.ports[port.URL] = ch
return ch, nil
}
type stateClient interface {
Send(*fnpb.StateRequest) error
Recv() (*fnpb.StateResponse, error)
}
// StateChannel manages state transactions over a single gRPC connection.
// It does not need to track readers and writers as carefully as the
// DataChannel, because the state protocol is request-based.
type StateChannel struct {
id string
client stateClient
requests chan *fnpb.StateRequest
nextRequestNo int32
responses map[string]chan<- *fnpb.StateResponse
mu sync.Mutex
// a closure that forces the state manager to recreate this stream.
forceRecreate func(id string, err error)
cancelFn context.CancelFunc
closedErr error
DoneCh <-chan struct{}
}
func (c *StateChannel) terminateStreamOnError(err error) {
c.mu.Lock()
if c.forceRecreate != nil {
c.closedErr = err
c.forceRecreate(c.id, err)
c.forceRecreate = nil
}
// Cancelling context after forcing recreation to ensure closedErr is set.
c.cancelFn()
c.mu.Unlock()
}
func newStateChannel(ctx context.Context, port exec.Port) (*StateChannel, error) {
ctx, cancelFn := context.WithCancel(ctx)
cc, err := dial(ctx, port.URL, 15*time.Second)
if err != nil {
cancelFn()
return nil, errors.Wrapf(err, "failed to connect to state service %v", port.URL)
}
client, err := fnpb.NewBeamFnStateClient(cc).State(ctx)
if err != nil {
cc.Close()
cancelFn()
return nil, errors.Wrapf(err, "failed to create state client %v", port.URL)
}
return makeStateChannel(ctx, cancelFn, port.URL, client), nil
}
func makeStateChannel(ctx context.Context, cancelFn context.CancelFunc, id string, client stateClient) *StateChannel {
ret := &StateChannel{
id: id,
client: client,
requests: make(chan *fnpb.StateRequest, 10),
responses: make(map[string]chan<- *fnpb.StateResponse),
cancelFn: cancelFn,
DoneCh: ctx.Done(),
}
go ret.read(ctx)
go ret.write(ctx)
return ret
}
func (c *StateChannel) read(ctx context.Context) {
for {
// Closing the context will have an error return from this call.
msg, err := c.client.Recv()
if err != nil {
c.terminateStreamOnError(err)
if err == io.EOF {
log.Warnf(ctx, "StateChannel[%v].read: closed", c.id)
return
}
log.Errorf(ctx, "StateChannel[%v].read bad: %v", c.id, err)
return
}
c.mu.Lock()
ch, ok := c.responses[msg.Id]
delete(c.responses, msg.Id)
c.mu.Unlock()
if !ok {
// This can happen if Send returns an error that write handles, but
// the message was actually sent.
log.Errorf(ctx, "StateChannel[%v].read: no consumer for state response: %v", c.id, proto.MarshalTextString(msg))
continue
}
select {
case ch <- msg:
// ok
default:
panic(fmt.Sprintf("StateChannel[%v].read: failed to consume state response: %v", c.id, proto.MarshalTextString(msg)))
}
}
}
func (c *StateChannel) write(ctx context.Context) {
var err error
var id string
for {
var req *fnpb.StateRequest
select {
case req = <-c.requests:
case <-c.DoneCh: // Close the goroutine on context cancel.
return
}
err = c.client.Send(req)
if err != nil {
id = req.Id
break // non-nil errors mean the stream is broken and can't be re-used.
}
}
if err == io.EOF {
log.Warnf(ctx, "StateChannel[%v].write EOF on send; fetching real error", c.id)
err = nil
for err == nil {
// Per GRPC stream documentation, if there's an EOF, we must call Recv
// until a non-nil error is returned, to ensure resources are cleaned up.
// https://godoc.org/google.golang.org/grpc#ClientConn.NewStream
_, err = c.client.Recv()
}
}
log.Errorf(ctx, "StateChannel[%v].write error on send: %v", c.id, err)
// Failed to send. Return error & unblock Send.
c.mu.Lock()
ch, ok := c.responses[id]
delete(c.responses, id)
c.mu.Unlock()
// Clean up everything else, this stream is done.
c.terminateStreamOnError(err)
if ok {
ch <- &fnpb.StateResponse{Id: id, Error: fmt.Sprintf("StateChannel[%v].write failed to send: %v", c.id, err)}
}
}
// Send sends a state request and returns the response.
func (c *StateChannel) Send(req *fnpb.StateRequest) (*fnpb.StateResponse, error) {
id := fmt.Sprintf("r%v", atomic.AddInt32(&c.nextRequestNo, 1))
req.Id = id
ch := make(chan *fnpb.StateResponse, 1)
c.mu.Lock()
if c.closedErr != nil {
defer c.mu.Unlock()
return nil, errors.Wrapf(c.closedErr, "StateChannel[%v].Send(%v): channel closed due to: %v", c.id, id, c.closedErr)
}
c.responses[id] = ch
c.mu.Unlock()
c.requests <- req
var resp *fnpb.StateResponse
select {
case resp = <-ch:
case <-c.DoneCh:
c.mu.Lock()
defer c.mu.Unlock()
return nil, errors.Wrapf(c.closedErr, "StateChannel[%v].Send(%v): context canceled", c.id, id)
}
if resp.Error != "" {
return nil, errors.New(resp.Error)
}
return resp, nil
}