-
Notifications
You must be signed in to change notification settings - Fork 153
/
transport.go
793 lines (695 loc) · 19.5 KB
/
transport.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
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
package execute
import (
"context"
"fmt"
"reflect"
"sync"
"sync/atomic"
"github.com/apache/arrow/go/v7/arrow/memory"
"github.com/influxdata/flux"
"github.com/influxdata/flux/arrow"
"github.com/influxdata/flux/codes"
"github.com/influxdata/flux/execute/table"
"github.com/influxdata/flux/internal/errors"
"github.com/influxdata/flux/internal/jaeger"
"github.com/influxdata/flux/interpreter"
"github.com/influxdata/flux/plan"
"github.com/opentracing/opentracing-go"
"github.com/opentracing/opentracing-go/log"
"go.uber.org/zap"
)
// Transport is an interface for handling raw messages.
type Transport interface {
// ProcessMessage will process a message in the Transport.
//
// Messages sent to the Transport may be one of many types.
// Known message should be handled as is appropriate, but
// unknown messages should be acked but otherwise ignored.
// An error should not be returned for unknown messages.
ProcessMessage(m Message) error
}
// AsyncTransport is a Transport that performs its work in a separate goroutine.
type AsyncTransport interface {
Transport
// Finished reports when the AsyncTransport has completed and there is no more work to do.
Finished() <-chan struct{}
// TransportProfile returns the profile for this transport.
// This is only valid after the channel returned by Finished is closed.
TransportProfile() flux.TransportProfile
}
var _ Transformation = (*consecutiveTransport)(nil)
// consecutiveTransport implements Transport by transporting data consecutively to the downstream Transformation.
type consecutiveTransport struct {
ctx context.Context
dispatcher Dispatcher
logger *zap.Logger
t Transport
messages MessageQueue
stack []interpreter.StackEntry
profile flux.TransportProfile
finished chan struct{}
errMu sync.Mutex
errValue error
schedulerState int32
inflight int32
totalMsgs int32
initSpanOnce sync.Once
span opentracing.Span
}
func newConsecutiveTransport(ctx context.Context, dispatcher Dispatcher, t Transformation, n plan.Node, logger *zap.Logger, mem memory.Allocator) *consecutiveTransport {
return &consecutiveTransport{
ctx: ctx,
dispatcher: dispatcher,
logger: logger,
t: WrapTransformationInTransport(t, mem),
// TODO(nathanielc): Have planner specify message queue initial buffer size.
messages: newMessageQueue(64),
profile: flux.TransportProfile{
NodeType: OperationType(t),
Label: string(n.ID()),
},
stack: n.CallStack(),
finished: make(chan struct{}),
}
}
func (t *consecutiveTransport) sourceInfo() string {
if len(t.stack) == 0 {
return ""
}
// Learn the filename from the bottom of the stack.
// We want the top most entry (deepest in the stack)
// from the primary file. We can retrieve the filename
// for the primary file by looking at the bottom of the
// stack and then finding the top-most entry with that
// filename.
filename := t.stack[len(t.stack)-1].Location.File
for i := 0; i < len(t.stack); i++ {
entry := t.stack[i]
if entry.Location.File == filename {
return fmt.Sprintf("@%s: %s", entry.Location, entry.FunctionName)
}
}
entry := t.stack[0]
return fmt.Sprintf("@%s: %s", entry.Location, entry.FunctionName)
}
func (t *consecutiveTransport) setErr(err error) {
t.errMu.Lock()
msg := "runtime error"
if srcInfo := t.sourceInfo(); srcInfo != "" {
msg += " " + srcInfo
}
err = errors.Wrap(err, codes.Inherit, msg)
t.errValue = err
t.errMu.Unlock()
}
func (t *consecutiveTransport) err() error {
t.errMu.Lock()
err := t.errValue
t.errMu.Unlock()
return err
}
func (t *consecutiveTransport) Finished() <-chan struct{} {
return t.finished
}
func (t *consecutiveTransport) TransportProfile() flux.TransportProfile {
return t.profile
}
func (t *consecutiveTransport) RetractTable(id DatasetID, key flux.GroupKey) error {
select {
case <-t.finished:
return t.err()
default:
}
t.pushMsg(&retractTableMsg{
srcMessage: srcMessage(id),
key: key,
})
return nil
}
func (t *consecutiveTransport) Process(id DatasetID, tbl flux.Table) error {
select {
case <-t.finished:
return t.err()
default:
}
t.pushMsg(&processMsg{
srcMessage: srcMessage(id),
table: newConsecutiveTransportTable(t, tbl),
})
return nil
}
func (t *consecutiveTransport) UpdateWatermark(id DatasetID, time Time) error {
select {
case <-t.finished:
return t.err()
default:
}
t.pushMsg(&updateWatermarkMsg{
srcMessage: srcMessage(id),
time: time,
})
return nil
}
func (t *consecutiveTransport) UpdateProcessingTime(id DatasetID, time Time) error {
select {
case <-t.finished:
return t.err()
default:
}
t.pushMsg(&updateProcessingTimeMsg{
srcMessage: srcMessage(id),
time: time,
})
return nil
}
func (t *consecutiveTransport) Finish(id DatasetID, err error) {
select {
case <-t.finished:
return
default:
}
t.pushMsg(&finishMsg{
srcMessage: srcMessage(id),
err: err,
})
}
func (t *consecutiveTransport) pushMsg(m Message) {
t.messages.Push(m)
atomic.AddInt32(&t.inflight, 1)
t.schedule()
}
func (t *consecutiveTransport) ProcessMessage(m Message) error {
t.pushMsg(m)
return nil
}
const (
// consecutiveTransport schedule states
idle int32 = iota
running
finished
)
// schedule indicates that there is work available to schedule.
func (t *consecutiveTransport) schedule() {
if t.tryTransition(idle, running) {
t.dispatcher.Schedule(t.processMessages)
}
}
// tryTransition attempts to transition into the new state and returns true on success.
func (t *consecutiveTransport) tryTransition(old, new int32) bool {
return atomic.CompareAndSwapInt32(&t.schedulerState, old, new)
}
// transition sets the new state.
func (t *consecutiveTransport) transition(new int32) {
atomic.StoreInt32(&t.schedulerState, new)
}
func (t *consecutiveTransport) initSpan(ctx context.Context) {
t.initSpanOnce.Do(func() {
t.span, _ = opentracing.StartSpanFromContext(ctx, t.profile.NodeType, opentracing.Tag{Key: "label", Value: t.profile.Label})
})
}
func (t *consecutiveTransport) finishSpan(err error) {
t.span.LogFields(log.Int("messages_processed", int(atomic.LoadInt32(&t.totalMsgs))), log.Error(err))
t.span.Finish()
}
func (t *consecutiveTransport) processMessages(ctx context.Context, throughput int) {
t.initSpan(ctx)
PROCESS:
i := 0
for m := t.messages.Pop(); m != nil; m = t.messages.Pop() {
atomic.AddInt32(&t.inflight, -1)
atomic.AddInt32(&t.totalMsgs, 1)
if f, err := t.processMessage(m); err != nil || f {
// Set the error if there was any
t.setErr(err)
// Transition to the finished state.
if t.tryTransition(running, finished) {
// Call Finish if we have not already
if !f {
m := &finishMsg{
srcMessage: srcMessage(m.SrcDatasetID()),
err: t.err(),
}
_ = t.t.ProcessMessage(m)
}
// We are finished
close(t.finished)
t.finishSpan(err)
return
}
}
i++
if i >= throughput {
// We have done enough work.
// Transition to the idle state and reschedule for later.
t.transition(idle)
t.schedule()
return
}
}
t.transition(idle)
// Check if more messages arrived after the above loop finished.
// This check must happen in the idle state.
if atomic.LoadInt32(&t.inflight) > 0 {
if t.tryTransition(idle, running) {
goto PROCESS
} // else we have already been scheduled again, we can return
}
}
// processMessage processes the message on t.
// The return value is true if the message was a FinishMsg.
func (t *consecutiveTransport) processMessage(m Message) (finished bool, err error) {
span := t.profile.StartSpan()
defer span.Finish()
if err := t.t.ProcessMessage(m); err != nil {
return false, err
}
finished = isFinishMessage(m)
return finished, nil
}
// Message is a message sent from one Dataset to another.
type Message interface {
// Type returns the MessageType for this Message.
Type() MessageType
// SrcDatasetID is the DatasetID that produced this Message.
SrcDatasetID() DatasetID
// Ack is used to acknowledge that the Message was received
// and terminated. A Message may be passed between various
// Transport implementations. When the Ack is received,
// this signals to the Message to release any memory it may
// have retained.
Ack()
// Dup is used to duplicate the Message.
// This is useful when the Message has to be sent to multiple
// receivers from a single sender.
Dup() Message
}
type MessageType int
const (
// RetractTableType is sent when the previous table for
// a given group key should be retracted.
RetractTableType MessageType = iota
// ProcessType is sent when there is an entire flux.Table
// ready to be processed from the upstream Dataset.
ProcessType
// UpdateWatermarkType is sent when there will be no more
// points older than the watermark for any key.
UpdateWatermarkType
// UpdateProcessingTimeType is sent to update the current time.
UpdateProcessingTimeType
// FinishType is sent when there are no more messages from
// the upstream Dataset or an upstream error occurred that
// caused the execution to abort.
FinishType
// ProcessChunkType is sent when a new table.Chunk is ready to
// be processed from the upstream Dataset.
ProcessChunkType
// FlushKeyType is sent when the upstream Dataset wishes
// to flush the data associated with a key presently stored
// in the Dataset.
FlushKeyType
)
type srcMessage DatasetID
func (m srcMessage) SrcDatasetID() DatasetID {
return DatasetID(m)
}
func (m srcMessage) Ack() {}
type RetractTableMsg interface {
Message
Key() flux.GroupKey
}
type retractTableMsg struct {
srcMessage
key flux.GroupKey
}
func (m *retractTableMsg) Type() MessageType {
return RetractTableType
}
func (m *retractTableMsg) Key() flux.GroupKey {
return m.key
}
func (m *retractTableMsg) Dup() Message {
return m
}
type ProcessMsg interface {
Message
Table() flux.Table
}
type processMsg struct {
srcMessage
table flux.Table
}
func (m *processMsg) Type() MessageType {
return ProcessType
}
func (m *processMsg) Table() flux.Table {
return m.table
}
func (m *processMsg) Ack() {
m.table.Done()
}
func (m *processMsg) Dup() Message {
cpy, _ := table.Copy(m.table)
m.table = cpy.Copy()
dup := *m
dup.table = cpy
return &dup
}
type UpdateWatermarkMsg interface {
Message
WatermarkTime() Time
}
type updateWatermarkMsg struct {
srcMessage
time Time
}
func (m *updateWatermarkMsg) Type() MessageType {
return UpdateWatermarkType
}
func (m *updateWatermarkMsg) WatermarkTime() Time {
return m.time
}
func (m *updateWatermarkMsg) Dup() Message {
return m
}
type UpdateProcessingTimeMsg interface {
Message
ProcessingTime() Time
}
type updateProcessingTimeMsg struct {
srcMessage
time Time
}
func (m *updateProcessingTimeMsg) Type() MessageType {
return UpdateProcessingTimeType
}
func (m *updateProcessingTimeMsg) ProcessingTime() Time {
return m.time
}
func (m *updateProcessingTimeMsg) Dup() Message {
return m
}
type FinishMsg interface {
Message
Error() error
}
type finishMsg struct {
srcMessage
err error
}
func (m *finishMsg) Type() MessageType {
return FinishType
}
func (m *finishMsg) Error() error {
return m.err
}
func (m *finishMsg) Dup() Message {
return m
}
// isFinishMessage will return true if the Message is a FinishMsg.
func isFinishMessage(m Message) bool {
_, ok := m.(FinishMsg)
return ok
}
type ProcessChunkMsg interface {
Message
TableChunk() table.Chunk
}
type processChunkMsg struct {
srcMessage
chunk table.Chunk
acked bool
}
func (m *processChunkMsg) Type() MessageType {
return ProcessChunkType
}
func (m *processChunkMsg) TableChunk() table.Chunk {
return m.chunk
}
func (m *processChunkMsg) Ack() {
if !m.acked {
m.chunk.Release()
m.chunk = table.Chunk{}
m.acked = true
}
}
func (m *processChunkMsg) Dup() Message {
dup := *m
if !dup.acked {
dup.chunk.Retain()
}
return &dup
}
type FlushKeyMsg interface {
Message
Key() flux.GroupKey
}
type flushKeyMsg struct {
srcMessage
key flux.GroupKey
}
func (m *flushKeyMsg) Type() MessageType {
return FlushKeyType
}
func (m *flushKeyMsg) Key() flux.GroupKey {
return m.key
}
func (m *flushKeyMsg) Dup() Message {
return m
}
// consecutiveTransportTable is a flux.Table that is being processed
// within a consecutiveTransport.
type consecutiveTransportTable struct {
transport *consecutiveTransport
tbl flux.Table
}
func newConsecutiveTransportTable(t *consecutiveTransport, tbl flux.Table) flux.Table {
return &consecutiveTransportTable{
transport: t,
tbl: tbl,
}
}
func (t *consecutiveTransportTable) Key() flux.GroupKey {
return t.tbl.Key()
}
func (t *consecutiveTransportTable) Cols() []flux.ColMeta {
return t.tbl.Cols()
}
func (t *consecutiveTransportTable) Do(f func(flux.ColReader) error) error {
return t.tbl.Do(func(cr flux.ColReader) error {
if err := t.validate(cr); err != nil {
fields := []zap.Field{
zap.String("source", t.transport.sourceInfo()),
zap.Error(err),
}
ctx, logger := t.transport.ctx, t.transport.logger
if span := opentracing.SpanFromContext(ctx); span != nil {
if traceID, sampled, found := jaeger.InfoFromSpan(span); found {
fields = append(fields,
zap.String("tracing/id", traceID),
zap.Bool("tracing/sampled", sampled),
)
}
}
logger.Info("Invalid column reader received from predecessor", fields...)
}
return f(cr)
})
}
func (t *consecutiveTransportTable) Done() {
t.tbl.Done()
}
func (t *consecutiveTransportTable) Empty() bool {
return t.tbl.Empty()
}
func (t *consecutiveTransportTable) validate(cr flux.ColReader) error {
if len(cr.Cols()) == 0 {
return nil
}
sz := table.Values(cr, 0).Len()
for i, n := 1, len(cr.Cols()); i < n; i++ {
nsz := table.Values(cr, i).Len()
if sz != nsz {
// Mismatched column lengths.
// Look at all column lengths so we can give a more complete
// error message.
// We avoid this in the usual case to avoid allocating an array
// of lengths for every table when it might not be needed.
lens := make(map[string]int, len(cr.Cols()))
for i, col := range cr.Cols() {
label := fmt.Sprintf("%s:%s", col.Label, col.Type)
lens[label] = table.Values(cr, i).Len()
}
return errors.Newf(codes.Internal, "mismatched column lengths: %v", lens)
}
}
return nil
}
// OperationType returns a string representation of the transformation
// operation represented by the Transport.
func OperationType(t interface{}) string {
if t, ok := t.(interface {
OperationType() string
}); ok {
return t.OperationType()
}
return reflect.TypeOf(t).String()
}
// transformationTransportAdapter will translate Message values sent to
// a Transport to an underlying Transformation.
type transformationTransportAdapter struct {
t Transformation
cache table.BuilderCache
}
// WrapTransformationInTransport will wrap a Transformation into
// a Transport to be used for the execution engine.
func WrapTransformationInTransport(t Transformation, mem memory.Allocator) Transport {
// If the Transformation implements the Transport interface,
// then we can just use that directly.
if tr, ok := t.(Transport); ok {
return tr
}
return &transformationTransportAdapter{
t: t,
cache: table.BuilderCache{
New: func(key flux.GroupKey) table.Builder {
return table.NewBufferedBuilder(key, mem)
},
},
}
}
func (t *transformationTransportAdapter) ProcessMessage(m Message) error {
switch m.Type() {
case RetractTableType:
m := m.(RetractTableMsg)
return t.t.RetractTable(m.SrcDatasetID(), m.Key())
case ProcessType:
m := m.(ProcessMsg)
return t.t.Process(m.SrcDatasetID(), m.Table())
case UpdateWatermarkType:
m := m.(UpdateWatermarkMsg)
return t.t.UpdateWatermark(m.SrcDatasetID(), m.WatermarkTime())
case UpdateProcessingTimeType:
m := m.(UpdateProcessingTimeMsg)
return t.t.UpdateProcessingTime(m.SrcDatasetID(), m.ProcessingTime())
case FinishType:
m := m.(FinishMsg)
// If there was an error, keep it.
err := m.Error()
if err == nil {
// If there are pending buffers that were never flushed,
// do that here. Do this only when an error didn't happen.
err = t.cache.ForEach(func(key flux.GroupKey, builder table.Builder) error {
table, err := builder.Table()
if err != nil {
return err
}
return t.t.Process(m.SrcDatasetID(), table)
})
}
t.t.Finish(m.SrcDatasetID(), err)
return nil
case ProcessChunkType:
defer m.Ack()
m := m.(ProcessChunkMsg)
// Retrieve the buffered builder and append the
// table view to it. The view is implemented using
// arrow.TableBuffer which is compatible with
// flux.ColReader so we can append it directly.
b, _ := table.GetBufferedBuilder(m.TableChunk().Key(), &t.cache)
buffer := m.TableChunk().Buffer()
return b.AppendBuffer(&buffer)
case FlushKeyType:
defer m.Ack()
m := m.(FlushKeyMsg)
// Retrieve the buffered builder for the given key
// and send the data to the next transformation.
tbl, ok, err := t.cache.Table(m.Key())
if err != nil || !ok {
return err
}
t.cache.ExpireTable(m.Key())
return t.t.Process(m.SrcDatasetID(), tbl)
default:
// Message is not handled by older Transformation implementations.
m.Ack()
return nil
}
}
func (t *transformationTransportAdapter) OperationType() string {
return OperationType(t.t)
}
var _ Transport = (*transportTransformationAdapter)(nil)
type transportTransformationAdapter struct {
Transport
}
// NewTransformationFromTransport will adapt a Transport to satisfy both
// the Transport and Transformation interfaces.
func NewTransformationFromTransport(t Transport) Transformation {
return &transportTransformationAdapter{Transport: t}
}
func (t *transportTransformationAdapter) ProcessMessage(m Message) error {
switch m := m.(type) {
case ProcessMsg:
defer m.Ack()
return t.Process(m.SrcDatasetID(), m.Table())
default:
return t.Transport.ProcessMessage(m)
}
}
// Process is implemented to remain compatible with legacy upstreams.
// It converts the incoming stream into a set of appropriate messages.
func (t *transportTransformationAdapter) Process(id DatasetID, tbl flux.Table) error {
if tbl.Empty() {
// Since the table is empty, it won't produce any column readers.
// Create an empty buffer which can be processed instead
// to force the creation of any potential state.
buffer := arrow.EmptyBuffer(tbl.Key(), tbl.Cols())
chunk := table.ChunkFromBuffer(buffer)
if err := t.processChunk(id, chunk); err != nil {
return err
}
} else {
if err := tbl.Do(func(cr flux.ColReader) error {
chunk := table.ChunkFromReader(cr)
chunk.Retain()
return t.processChunk(id, chunk)
}); err != nil {
return err
}
}
return t.flushKey(id, tbl.Key())
}
func (t *transportTransformationAdapter) processChunk(id DatasetID, chunk table.Chunk) error {
m := processChunkMsg{
srcMessage: srcMessage(id),
chunk: chunk,
}
return t.Transport.ProcessMessage(&m)
}
func (t *transportTransformationAdapter) flushKey(id DatasetID, key flux.GroupKey) error {
m := flushKeyMsg{
srcMessage: srcMessage(id),
key: key,
}
return t.Transport.ProcessMessage(&m)
}
// Finish is implemented to remain compatible with legacy upstreams.
func (t *transportTransformationAdapter) Finish(id DatasetID, err error) {
m := finishMsg{
srcMessage: srcMessage(id),
err: err,
}
_ = t.Transport.ProcessMessage(&m)
}
func (t *transportTransformationAdapter) OperationType() string {
return OperationType(t.Transport)
}
func (t *transportTransformationAdapter) RetractTable(_ DatasetID, _ flux.GroupKey) error {
return nil
}
func (t *transportTransformationAdapter) UpdateWatermark(_ DatasetID, _ Time) error {
return nil
}
func (t *transportTransformationAdapter) UpdateProcessingTime(_ DatasetID, _ Time) error {
return nil
}