-
Notifications
You must be signed in to change notification settings - Fork 152
/
transport.go
448 lines (391 loc) · 10 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
package execute
import (
"context"
"fmt"
"reflect"
"sync"
"sync/atomic"
"github.com/influxdata/flux"
"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"
"go.uber.org/zap"
)
type Transport interface {
Transformation
// Finished reports when the Transport has completed and there is no more work to do.
Finished() <-chan struct{}
}
// consecutiveTransport implements Transport by transporting data consecutively to the downstream Transformation.
type consecutiveTransport struct {
ctx context.Context
dispatcher Dispatcher
logger *zap.Logger
t Transformation
messages MessageQueue
stack []interpreter.StackEntry
finished chan struct{}
errMu sync.Mutex
errValue error
schedulerState int32
inflight int32
}
func newConsecutiveTransport(ctx context.Context, dispatcher Dispatcher, t Transformation, n plan.Node, logger *zap.Logger) *consecutiveTransport {
return &consecutiveTransport{
ctx: ctx,
dispatcher: dispatcher,
logger: logger,
t: t,
// TODO(nathanielc): Have planner specify message queue initial buffer size.
messages: newMessageQueue(64),
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) 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()
}
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) processMessages(ctx context.Context, throughput int) {
PROCESS:
i := 0
for m := t.messages.Pop(); m != nil; m = t.messages.Pop() {
atomic.AddInt32(&t.inflight, -1)
if f, err := processMessage(ctx, t.t, 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 {
t.t.Finish(m.SrcDatasetID(), t.err())
}
// We are finished
close(t.finished)
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
}
}
func (t *consecutiveTransport) Label() string {
return t.t.Label()
}
func (t *consecutiveTransport) SetLabel(label string) {
t.t.SetLabel(label)
}
// processMessage processes the message on t.
// The return value is true if the message was a FinishMsg.
func processMessage(ctx context.Context, t Transformation, m Message) (finished bool, err error) {
switch m := m.(type) {
case RetractTableMsg:
err = t.RetractTable(m.SrcDatasetID(), m.Key())
case ProcessMsg:
b := m.Table()
_, span := StartSpanFromContext(ctx, reflect.TypeOf(t).String(), t.Label())
err = t.Process(m.SrcDatasetID(), b)
if span != nil {
span.Finish()
}
case UpdateWatermarkMsg:
err = t.UpdateWatermark(m.SrcDatasetID(), m.WatermarkTime())
case UpdateProcessingTimeMsg:
err = t.UpdateProcessingTime(m.SrcDatasetID(), m.ProcessingTime())
case FinishMsg:
t.Finish(m.SrcDatasetID(), m.Error())
finished = true
}
return
}
type Message interface {
Type() MessageType
SrcDatasetID() DatasetID
}
type MessageType int
const (
RetractTableType MessageType = iota
ProcessType
UpdateWatermarkType
UpdateProcessingTimeType
FinishType
)
type srcMessage DatasetID
func (m srcMessage) SrcDatasetID() DatasetID {
return DatasetID(m)
}
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
}
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
}
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
}
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
}
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
}
// 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
}