-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
datamgr.go
712 lines (623 loc) · 21.1 KB
/
datamgr.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
// 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/v2/go/pkg/beam/core/runtime/exec"
"github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors"
"github.com/apache/beam/sdks/v2/go/pkg/beam/log"
fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1"
)
const (
chunkSize = int(4e6) // Bytes to put in a single gRPC message. Max is slightly higher.
bufElements = 20 // Number of chunks buffered per reader.
)
// ScopedDataManager scopes the global gRPC data manager to a single instruction.
// The indirection makes it easier to control access.
type ScopedDataManager struct {
mgr *DataChannelManager
instID instructionID
openPorts []exec.Port
closed bool
mu sync.Mutex
}
// NewScopedDataManager returns a ScopedDataManager for the given instruction.
func NewScopedDataManager(mgr *DataChannelManager, instID instructionID) *ScopedDataManager {
return &ScopedDataManager{mgr: mgr, instID: instID}
}
// OpenWrite opens an io.WriteCloser on the given stream.
func (s *ScopedDataManager) OpenWrite(ctx context.Context, id exec.StreamID) (io.WriteCloser, error) {
ch, err := s.open(ctx, id.Port)
if err != nil {
return nil, err
}
return ch.OpenWrite(ctx, id.PtransformID, s.instID), nil
}
// OpenElementChan returns a channel of exec.Elements on the given stream.
func (s *ScopedDataManager) OpenElementChan(ctx context.Context, id exec.StreamID, expectedTimerTransforms []string) (<-chan exec.Elements, error) {
ch, err := s.open(ctx, id.Port)
if err != nil {
return nil, err
}
return ch.OpenElementChan(ctx, id.PtransformID, s.instID, expectedTimerTransforms)
}
// OpenTimerWrite opens an io.WriteCloser on the given stream to write timers
func (s *ScopedDataManager) OpenTimerWrite(ctx context.Context, id exec.StreamID, family string) (io.WriteCloser, error) {
ch, err := s.open(ctx, id.Port)
if err != nil {
return nil, err
}
return ch.OpenTimerWrite(ctx, id.PtransformID, s.instID, family), nil
}
func (s *ScopedDataManager) open(ctx context.Context, port exec.Port) (*DataChannel, error) {
s.mu.Lock()
if s.closed {
s.mu.Unlock()
return nil, errors.Errorf("instruction %v no longer processing", s.instID)
}
s.openPorts = append(s.openPorts, port)
local := s.mgr
s.mu.Unlock()
return local.Open(ctx, port) // don't hold lock over potentially slow operation
}
// Close prevents new IO for this instruction.
func (s *ScopedDataManager) Close() error {
s.mu.Lock()
defer s.mu.Unlock()
s.closed = true
err := s.mgr.closeInstruction(s.instID, s.openPorts)
s.mgr = nil
return err
}
// DataChannelManager manages data channels over the Data API. A fixed number of channels
// are generally used, each managing multiple logical byte streams. Thread-safe.
type DataChannelManager struct {
ports map[string]*DataChannel
mu sync.Mutex // guards the ports map
}
// Open opens a R/W DataChannel over the given port.
func (m *DataChannelManager) Open(ctx context.Context, port exec.Port) (*DataChannel, error) {
if port.URL == "" {
panic("empty port")
}
m.mu.Lock()
defer m.mu.Unlock()
if m.ports == nil {
m.ports = make(map[string]*DataChannel)
}
if con, ok := m.ports[port.URL]; ok {
return con, nil
}
ch, err := newDataChannel(ctx, port)
if err != nil {
return nil, err
}
ch.forceRecreate = func(id string, err error) {
log.Warnf(ctx, "forcing DataChannel[%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
}
func (m *DataChannelManager) closeInstruction(instID instructionID, ports []exec.Port) error {
m.mu.Lock()
defer m.mu.Unlock()
var firstNonNilError error
for _, port := range ports {
ch, ok := m.ports[port.URL]
if !ok {
continue
}
err := ch.removeInstruction(instID)
if err != nil && firstNonNilError == nil {
firstNonNilError = err
}
}
return firstNonNilError
}
// clientID identifies a client of a connected channel.
type clientID struct {
instID instructionID
ptransformID string
}
// This is a reduced version of the full gRPC interface to help with testing.
type dataClient interface {
Send(*fnpb.Elements) error
Recv() (*fnpb.Elements, error)
}
var _ dataClient = (fnpb.BeamFnData_DataClient)(nil) // Assert our interfaces are compatible.
// DataChannel manages a single gRPC stream over the Data API. Data from
// multiple bundles can be multiplexed over this stream. Data is pushed
// over the channel, so data for a reader may arrive before the reader
// connects.
// Thread-safe.
type DataChannel struct {
id string
client dataClient
writers map[instructionID]map[string]*dataWriter // PTransformID
timerWriters map[instructionID]map[timerKey]*timerWriter
channels map[instructionID]*elementsChan
// recently terminated instructions
endedInstructions map[instructionID]struct{}
rmQueue []instructionID
// readErr indicates a client.Recv error and is used to prevent new readers.
readErr error
// a closure that forces the data manager to recreate this stream.
forceRecreate func(id string, err error)
cancelFn context.CancelFunc // Allows writers to stop the grpc reading goroutine.
mu sync.Mutex // guards mutable internal data, notably the maps and readErr.
}
type timerKey struct {
ptransformID, family string
}
// elementsChan abstracts the management for this instruction's channel.
//
// The only runner signal that all data for an instruction has been received
// is when the ch channel has been closed. However, we may receive all data
// before the instruction begins consuming it, and there may be multiple PTransforms
// in the instruction that may need data through this channel. Until the instruction
// arrives, received data needs to be cached, and we cannot close the channel.
//
// The channel may only close if the want == got and want > 0.
// want is set once when the Source requests it.
// got is incremented only if we receive an IsLast signal for a given
// instruction/transform pair.
type elementsChan struct {
closed uint32 // Closed if != 0
instID instructionID
mu sync.Mutex
want, got int32
ch chan exec.Elements // must only be closed by the read loop
done chan struct{} // Forces escape from a blocked write to allow channel close.
}
// InstructionEnded signals the read loop to close the channel.
func (ec *elementsChan) InstructionEnded() {
close(ec.done)
}
// Closed indicates if all expected streams are complete
func (ec *elementsChan) Closed() bool {
return atomic.LoadUint32(&ec.closed) != 0
}
// PTransformDone signals that a PTransform has no more data coming to it.
// If permitted, PTransformDone closes the channel.
func (ec *elementsChan) PTransformDone() {
ec.mu.Lock()
defer ec.mu.Unlock()
ec.got++
if ec.want > 0 && ec.want == ec.got {
if !ec.Closed() {
atomic.StoreUint32(&ec.closed, 1)
close(ec.ch)
}
}
}
func newDataChannel(ctx context.Context, port exec.Port) (*DataChannel, error) {
ctx, cancelFn := context.WithCancel(ctx)
cc, err := dial(ctx, port.URL, "data", 15*time.Second)
if err != nil {
cancelFn()
return nil, errors.Wrapf(err, "failed to connect to data service at %v", port.URL)
}
client, err := fnpb.NewBeamFnDataClient(cc).Data(ctx)
if err != nil {
cc.Close()
cancelFn()
return nil, errors.Wrapf(err, "failed to create data client on %v", port.URL)
}
return makeDataChannel(ctx, port.URL, client, func() {
cc.Close()
cancelFn()
}), nil
}
func makeDataChannel(ctx context.Context, id string, client dataClient, cancelFn context.CancelFunc) *DataChannel {
ret := &DataChannel{
id: id,
client: client,
writers: make(map[instructionID]map[string]*dataWriter),
timerWriters: make(map[instructionID]map[timerKey]*timerWriter),
channels: make(map[instructionID]*elementsChan),
endedInstructions: make(map[instructionID]struct{}),
cancelFn: cancelFn,
}
go ret.read(ctx)
return ret
}
// terminateStreamOnError requires the lock to be held.
func (c *DataChannel) terminateStreamOnError(err error) {
c.cancelFn() // A context.CancelFunc is threadsafe and indempotent.
if c.forceRecreate != nil {
c.forceRecreate(c.id, err)
c.forceRecreate = nil
}
}
// OpenWrite returns an io.WriteCloser of the data elements for the given instruction and ptransform.
func (c *DataChannel) OpenWrite(ctx context.Context, ptransformID string, instID instructionID) io.WriteCloser {
return c.makeWriter(ctx, clientID{ptransformID: ptransformID, instID: instID})
}
// OpenElementChan returns a channel of typex.Elements for the given instruction and ptransform.
func (c *DataChannel) OpenElementChan(ctx context.Context, ptransformID string, instID instructionID, expectedTimerTransforms []string) (<-chan exec.Elements, error) {
c.mu.Lock()
defer c.mu.Unlock()
cid := clientID{ptransformID: ptransformID, instID: instID}
if c.readErr != nil {
return nil, fmt.Errorf("opening a reader %v on a closed channel. Original error: %w", cid, c.readErr)
}
return c.makeChannel(true, cid, expectedTimerTransforms...).ch, nil
}
// makeChannel creates a channel of exec.Elements. It expects to be called while c.mu is held.
func (c *DataChannel) makeChannel(fromSource bool, id clientID, additionalTransforms ...string) *elementsChan {
if ec, ok := c.channels[id.instID]; ok {
ec.mu.Lock()
defer ec.mu.Unlock()
if fromSource {
ec.want = (1 + int32(len(additionalTransforms)))
}
if _, ok := c.endedInstructions[id.instID]; ok || (ec.want > 0 && ec.want == ec.got) {
atomic.StoreUint32(&ec.closed, 1)
close(ec.ch)
}
return ec
}
ec := &elementsChan{
instID: id.instID,
ch: make(chan exec.Elements, 20),
done: make(chan struct{}),
}
if fromSource {
ec.want = 1 + int32(len(additionalTransforms))
}
// Just in case initial data for an instruction arrives *after* an instructon has ended.
// eg. it was blocked by another reader being slow, or the other instruction failed.
// So we provide a pre-completed reader, and do not cache it, as there's no further cleanup for it.
if _, ok := c.endedInstructions[id.instID]; ok {
// Since this is freshly created, we can set the close conditions immeadiately.
atomic.StoreUint32(&ec.closed, 1)
close(ec.ch)
return ec
}
c.channels[id.instID] = ec
return ec
}
// OpenTimerWrite returns io.WriteCloser for the given timerFamilyID, instruction and ptransform.
func (c *DataChannel) OpenTimerWrite(ctx context.Context, ptransformID string, instID instructionID, family string) io.WriteCloser {
return c.makeTimerWriter(ctx, clientID{ptransformID: ptransformID, instID: instID}, family)
}
func (c *DataChannel) read(ctx context.Context) {
cache := make(map[instructionID]*elementsChan)
seenLast := make([]clientID, 0, 5)
for {
msg, err := c.client.Recv()
if err != nil {
// This connection is bad, so we should close and delete all extant streams.
c.mu.Lock()
c.readErr = err // prevent not yet opened readers from hanging.
// Readers must be closed from this goroutine, since we can't
// close the elementsChan channel twice, or send on those closed channels.
// Any other approach is racy, and may cause one of the above panics.
for instID, ec := range c.channels {
if !ec.Closed() {
atomic.StoreUint32(&ec.closed, 1)
close(ec.ch)
}
delete(cache, instID)
}
c.terminateStreamOnError(err)
c.mu.Unlock()
if err == io.EOF {
return
}
log.Errorf(ctx, "DataChannel.read %v bad: %v", c.id, err)
return
}
// Each message may contain segments for multiple streams, so we
// must treat each segment in isolation. We maintain a local cache
// to reduce lock contention.
iterateElements(c, cache, &seenLast, msg.GetTimers(),
func(elm *fnpb.Elements_Timers) exec.Elements {
return exec.Elements{Timers: elm.GetTimers(), PtransformID: elm.GetTransformId(), TimerFamilyID: elm.GetTimerFamilyId()}
})
iterateElements(c, cache, &seenLast, msg.GetData(),
func(elm *fnpb.Elements_Data) exec.Elements {
return exec.Elements{Data: elm.GetData(), PtransformID: elm.GetTransformId()}
})
// Mark all readers that we've seen the last of as done, after queuing their elements.
if len(seenLast) > 0 {
c.mu.Lock()
for _, id := range seenLast {
r, ok := cache[id.instID]
if !ok {
continue // we've already closed this cached reader, skip
}
r.PTransformDone()
if r.Closed() {
// Clean up local bookkeeping. We'll never see another message
// for it again. We have to be careful not to remove the real
// one, because readers may be initialized after we've seen
// the full stream.
delete(cache, id.instID)
}
}
seenLast = seenLast[:0] // reset for re-use
c.mu.Unlock()
}
}
}
// dataEle is a light interface against the proto Data and Timer Elements.
type dataEle interface {
GetTransformId() string
GetInstructionId() string
GetIsLast() bool
}
func iterateElements[E dataEle](c *DataChannel, cache map[instructionID]*elementsChan, seenLast *[]clientID, elms []E, wrap func(E) exec.Elements) {
for _, elm := range elms {
id := clientID{ptransformID: elm.GetTransformId(), instID: instructionID(elm.GetInstructionId())}
var ec *elementsChan
if local, ok := cache[id.instID]; ok {
ec = local
} else {
c.mu.Lock()
ec = c.makeChannel(false, id)
c.mu.Unlock()
cache[id.instID] = ec
}
if ec.Closed() {
continue
}
// This send deliberately blocks if we exceed the buffering for
// a reader. We can't buffer the entire main input, if some user code
// is slow (or gets stuck). If the local side closes, the reader
// will be marked as completed and further remote data will be ignored.
select {
case ec.ch <- wrap(elm):
case <-ec.done: // In case of out of band cancels.
ec.mu.Lock()
atomic.StoreUint32(&ec.closed, 1)
close(ec.ch)
ec.mu.Unlock()
}
if elm.GetIsLast() {
*seenLast = append(*seenLast, id)
}
}
}
const endedInstructionCap = 32
// removeInstruction closes all readers and writers registered for the instruction
// and deletes this instruction from the channel's reader and writer maps.
func (c *DataChannel) removeInstruction(instID instructionID) error {
c.mu.Lock()
// We don't want to leak memory, so cap the endedInstructions list.
if len(c.rmQueue) >= endedInstructionCap {
toRemove := c.rmQueue[0]
c.rmQueue = c.rmQueue[1:]
delete(c.endedInstructions, toRemove)
}
c.endedInstructions[instID] = struct{}{}
c.rmQueue = append(c.rmQueue, instID)
ws := c.writers[instID]
tws := c.timerWriters[instID]
ec := c.channels[instID]
// Prevent other users while we iterate.
delete(c.writers, instID)
delete(c.timerWriters, instID)
delete(c.channels, instID)
// Return readErr to defend against data loss via short reads.
err := c.readErr
c.mu.Unlock()
for _, w := range ws {
w.Close()
}
for _, tw := range tws {
tw.Close()
}
if ec != nil {
ec.InstructionEnded()
}
return err
}
func (c *DataChannel) makeWriter(ctx context.Context, id clientID) *dataWriter {
c.mu.Lock()
defer c.mu.Unlock()
var m map[string]*dataWriter
var ok bool
if m, ok = c.writers[id.instID]; !ok {
m = make(map[string]*dataWriter)
c.writers[id.instID] = m
}
if w, ok := m[id.ptransformID]; ok {
return w
}
// We don't check for ended instructions for writers, as writers
// can only be created if an instruction is in scope, and aren't
// runner or user directed.
w := &dataWriter{ch: c, id: id}
m[id.ptransformID] = w
return w
}
type dataWriter struct {
buf []byte
id clientID
ch *DataChannel
}
// send requires the ch.mu lock to be held.
func (w *dataWriter) send(msg *fnpb.Elements) error {
if err := w.ch.client.Send(msg); err != nil {
if err == io.EOF {
log.Warnf(context.TODO(), "dataWriter[%v;%v] EOF on send; fetching real error", w.id, w.ch.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://pkg.go.dev/google.golang.org/grpc#ClientConn.NewStream
_, err = w.ch.client.Recv()
}
}
log.Warnf(context.TODO(), "dataWriter[%v;%v] error on send: %v", w.id, w.ch.id, err)
w.ch.terminateStreamOnError(err)
return err
}
return nil
}
func (w *dataWriter) Close() error {
// Don't acquire the locks as Flush will do so.
l := len(w.buf)
err := w.Flush()
if err != nil {
return errors.Wrapf(err, "dataWriter[%v;%v].Close: error flushing buffer of length %d", w.id, w.ch.id, l)
}
// TODO(BEAM-13082): Consider a sync.Pool to reuse < 64MB buffers.
// The dataWriter won't be reused, but may be referenced elsewhere.
// Drop the buffer to let it be GC'd.
w.buf = nil
// Now acquire the locks since we're sending.
w.ch.mu.Lock()
defer w.ch.mu.Unlock()
delete(w.ch.writers[w.id.instID], w.id.ptransformID)
msg := &fnpb.Elements{
Data: []*fnpb.Elements_Data{
{
InstructionId: string(w.id.instID),
TransformId: w.id.ptransformID,
// TODO(https://github.com/apache/beam/issues/21164): Set IsLast true on final flush instead of w/empty sentinel?
// Empty data == sentinel
IsLast: true,
},
},
}
return w.send(msg)
}
const largeBufferNotificationThreshold = 1024 * 1024 * 1024 // 1GB
func (w *dataWriter) Flush() error {
if w.buf == nil {
return nil
}
w.ch.mu.Lock()
defer w.ch.mu.Unlock()
msg := &fnpb.Elements{
Data: []*fnpb.Elements_Data{
{
InstructionId: string(w.id.instID),
TransformId: w.id.ptransformID,
Data: w.buf,
},
},
}
if l := len(w.buf); l > largeBufferNotificationThreshold {
log.Infof(context.TODO(), "dataWriter[%v;%v].Flush flushed large buffer of length %d", w.id, w.ch.id, l)
}
w.buf = w.buf[:0]
return w.send(msg)
}
func (w *dataWriter) Write(p []byte) (n int, err error) {
if len(w.buf)+len(p) > chunkSize {
l := len(w.buf)
// We can't fit this message into the buffer. We need to flush the buffer
if err := w.Flush(); err != nil {
return 0, errors.Wrapf(err, "datamgr.go [%v]: error flushing buffer of length %d", w.id, l)
}
}
// At this point there's room in the buffer one way or another.
w.buf = append(w.buf, p...)
return len(p), nil
}
func (c *DataChannel) makeTimerWriter(ctx context.Context, id clientID, family string) *timerWriter {
c.mu.Lock()
defer c.mu.Unlock()
var m map[timerKey]*timerWriter
var ok bool
if m, ok = c.timerWriters[id.instID]; !ok {
m = make(map[timerKey]*timerWriter)
c.timerWriters[id.instID] = m
}
tk := timerKey{ptransformID: id.ptransformID, family: family}
if w, ok := m[tk]; ok {
return w
}
// We don't check for finished instructions for writers, as writers
// can only be created if an instruction is in scope, and aren't
// runner or user directed.
w := &timerWriter{ch: c, id: id, timerFamilyID: family}
m[tk] = w
return w
}
type timerWriter struct {
id clientID
timerFamilyID string
ch *DataChannel
}
// send requires the ch.mu lock to be held.
func (w *timerWriter) send(msg *fnpb.Elements) error {
if err := w.ch.client.Send(msg); err != nil {
if err == io.EOF {
log.Warnf(context.TODO(), "timerWriter[%v;%v] EOF on send; fetching real error", w.id, w.ch.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://pkg.go.dev/google.golang.org/grpc#ClientConn.NewStream
_, err = w.ch.client.Recv()
}
}
log.Warnf(context.TODO(), "timerWriter[%v;%v] error on send: %v", w.id, w.ch.id, err)
w.ch.terminateStreamOnError(err)
return err
}
return nil
}
func (w *timerWriter) Close() error {
w.ch.mu.Lock()
defer w.ch.mu.Unlock()
delete(w.ch.timerWriters[w.id.instID], timerKey{w.id.ptransformID, w.timerFamilyID})
var msg *fnpb.Elements
msg = &fnpb.Elements{
Timers: []*fnpb.Elements_Timers{
{
InstructionId: string(w.id.instID),
TransformId: w.id.ptransformID,
TimerFamilyId: w.timerFamilyID,
IsLast: true,
},
},
}
return w.send(msg)
}
func (w *timerWriter) writeTimers(p []byte) error {
w.ch.mu.Lock()
defer w.ch.mu.Unlock()
msg := &fnpb.Elements{
Timers: []*fnpb.Elements_Timers{
{
InstructionId: string(w.id.instID),
TransformId: w.id.ptransformID,
TimerFamilyId: w.timerFamilyID,
Timers: p,
},
},
}
return w.send(msg)
}
func (w *timerWriter) Write(p []byte) (n int, err error) {
// write timers directly without buffering.
if err := w.writeTimers(p); err != nil {
return 0, err
}
return len(p), nil
}