-
Notifications
You must be signed in to change notification settings - Fork 153
/
executor.go
349 lines (290 loc) · 8.17 KB
/
executor.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
// Package execute contains the implementation of the execution phase in the query engine.
package execute
import (
"context"
"fmt"
"runtime/debug"
"sync"
"time"
"github.com/influxdata/flux"
"github.com/influxdata/flux/memory"
"github.com/influxdata/flux/plan"
"github.com/pkg/errors"
"go.uber.org/zap"
"go.uber.org/zap/zapcore"
)
type Executor interface {
// Execute will begin execution of the plan.Spec using the memory allocator.
// This returns a mapping of names to the query results.
// This will also return a channel for the Metadata from the query. The channel
// may return zero or more values. The returned channel must not require itself to
// be read so the executor must allocate enough space in the channel so if the channel
// is unread that it will not block.
Execute(ctx context.Context, p *plan.Spec, a *memory.Allocator) (map[string]flux.Result, <-chan flux.Metadata, error)
}
type executor struct {
deps Dependencies
logger *zap.Logger
}
func NewExecutor(deps Dependencies, logger *zap.Logger) Executor {
if logger == nil {
logger = zap.NewNop()
}
e := &executor{
deps: deps,
logger: logger,
}
return e
}
type streamContext struct {
bounds *Bounds
}
func (ctx streamContext) Bounds() *Bounds {
return ctx.bounds
}
type executionState struct {
p *plan.Spec
deps Dependencies
alloc *memory.Allocator
resources flux.ResourceManagement
results map[string]flux.Result
sources []Source
metaCh chan flux.Metadata
transports []Transport
dispatcher *poolDispatcher
logger *zap.Logger
}
func (e *executor) Execute(ctx context.Context, p *plan.Spec, a *memory.Allocator) (map[string]flux.Result, <-chan flux.Metadata, error) {
es, err := e.createExecutionState(ctx, p, a)
if err != nil {
return nil, nil, errors.Wrap(err, "failed to initialize execute state")
}
es.logger = e.logger
es.do(ctx)
return es.results, es.metaCh, nil
}
func validatePlan(p *plan.Spec) error {
if p.Resources.ConcurrencyQuota == 0 {
return errors.New("plan must have a non-zero concurrency quota")
}
return nil
}
func (e *executor) createExecutionState(ctx context.Context, p *plan.Spec, a *memory.Allocator) (*executionState, error) {
if err := validatePlan(p); err != nil {
return nil, errors.Wrap(err, "invalid plan")
}
es := &executionState{
p: p,
deps: e.deps,
alloc: a,
resources: p.Resources,
results: make(map[string]flux.Result),
// TODO(nathanielc): Have the planner specify the dispatcher throughput
dispatcher: newPoolDispatcher(10, e.logger),
}
v := &createExecutionNodeVisitor{
ctx: ctx,
es: es,
nodes: make(map[plan.Node]Node),
}
if err := p.BottomUpWalk(v.Visit); err != nil {
return nil, err
}
// Only sources can be a MetadataNode at the moment so allocate enough
// space for all of them to report metadata. Not all of them will necessarily
// report metadata.
es.metaCh = make(chan flux.Metadata, len(es.sources))
return v.es, nil
}
// createExecutionNodeVisitor visits each node in a physical query plan
// and creates a node responsible for executing that physical operation.
type createExecutionNodeVisitor struct {
ctx context.Context
es *executionState
nodes map[plan.Node]Node
}
func skipYields(pn plan.Node) plan.Node {
isYield := func(pn plan.Node) bool {
_, ok := pn.ProcedureSpec().(plan.YieldProcedureSpec)
return ok
}
for isYield(pn) {
pn = pn.Predecessors()[0]
}
return pn
}
func nonYieldPredecessors(pn plan.Node) []plan.Node {
nodes := make([]plan.Node, len(pn.Predecessors()))
for i, pred := range pn.Predecessors() {
nodes[i] = skipYields(pred)
}
return nodes
}
// Visit creates the node that will execute a particular plan node
func (v *createExecutionNodeVisitor) Visit(node plan.Node) error {
ppn, ok := node.(*plan.PhysicalPlanNode)
if !ok {
return fmt.Errorf("cannot execute plan node of type %T", node)
}
spec := node.ProcedureSpec()
kind := spec.Kind()
id := DatasetIDFromNodeID(node.ID())
if yieldSpec, ok := spec.(plan.YieldProcedureSpec); ok {
r := newResult(yieldSpec.YieldName())
v.es.results[yieldSpec.YieldName()] = r
v.nodes[skipYields(node)].AddTransformation(r)
return nil
}
// Add explicit stream context if bounds are set on this node
var streamContext streamContext
if node.Bounds() != nil {
streamContext.bounds = &Bounds{
Start: node.Bounds().Start,
Stop: node.Bounds().Stop,
}
}
// Build execution context
ec := executionContext{
ctx: v.ctx,
es: v.es,
parents: make([]DatasetID, len(node.Predecessors())),
streamContext: streamContext,
}
for i, pred := range nonYieldPredecessors(node) {
ec.parents[i] = DatasetIDFromNodeID(pred.ID())
}
// If node is a leaf, create a source
if len(node.Predecessors()) == 0 {
createSourceFn, ok := procedureToSource[kind]
if !ok {
return fmt.Errorf("unsupported source kind %v", kind)
}
source, err := createSourceFn(spec, id, ec)
if err != nil {
return err
}
v.es.sources = append(v.es.sources, source)
v.nodes[node] = source
} else {
// If node is internal, create a transformation.
// For each predecessor, add a transport for sending data upstream.
createTransformationFn, ok := procedureToTransformation[kind]
if !ok {
return fmt.Errorf("unsupported procedure %v", kind)
}
tr, ds, err := createTransformationFn(id, AccumulatingMode, spec, ec)
if err != nil {
return err
}
if ppn.TriggerSpec == nil {
ppn.TriggerSpec = plan.DefaultTriggerSpec
}
ds.SetTriggerSpec(ppn.TriggerSpec)
v.nodes[node] = ds
for _, p := range nonYieldPredecessors(node) {
executionNode := v.nodes[p]
transport := newConsecutiveTransport(v.es.dispatcher, tr)
v.es.transports = append(v.es.transports, transport)
executionNode.AddTransformation(transport)
}
if plan.HasSideEffect(spec) && len(node.Successors()) == 0 {
name := string(node.ID())
r := newResult(name)
v.es.results[name] = r
v.nodes[skipYields(node)].AddTransformation(r)
}
}
return nil
}
func (es *executionState) abort(err error) {
for _, r := range es.results {
r.(*result).abort(err)
}
}
func (es *executionState) do(ctx context.Context) {
var wg sync.WaitGroup
for _, src := range es.sources {
wg.Add(1)
go func(src Source) {
defer wg.Done()
// Setup panic handling on the source goroutines
defer func() {
if e := recover(); e != nil {
// We had a panic, abort the entire execution.
var err error
switch e := e.(type) {
case error:
err = e
default:
err = fmt.Errorf("%v", e)
}
if _, ok := err.(memory.LimitExceededError); ok {
es.abort(err)
return
}
es.abort(fmt.Errorf("panic: %v", err))
if entry := es.logger.Check(zapcore.InfoLevel, "Execute source panic"); entry != nil {
entry.Stack = string(debug.Stack())
entry.Write(zap.Error(err))
}
}
}()
src.Run(ctx)
if mdn, ok := src.(MetadataNode); ok {
es.metaCh <- mdn.Metadata()
}
}(src)
}
go func() {
defer close(es.metaCh)
wg.Wait()
}()
es.dispatcher.Start(es.resources.ConcurrencyQuota, ctx)
go func() {
// Wait for all transports to finish
for _, t := range es.transports {
select {
case <-t.Finished():
case <-ctx.Done():
es.abort(ctx.Err())
case err := <-es.dispatcher.Err():
if err != nil {
es.abort(err)
}
}
}
// Check for any errors on the dispatcher
err := es.dispatcher.Stop()
if err != nil {
es.abort(err)
}
}()
}
// Need a unique stream context per execution context
type executionContext struct {
ctx context.Context
es *executionState
parents []DatasetID
streamContext streamContext
}
func resolveTime(qt flux.Time, now time.Time) Time {
return Time(qt.Time(now).UnixNano())
}
func (ec executionContext) Context() context.Context {
return ec.ctx
}
func (ec executionContext) ResolveTime(qt flux.Time) Time {
return resolveTime(qt, ec.es.p.Now)
}
func (ec executionContext) StreamContext() StreamContext {
return ec.streamContext
}
func (ec executionContext) Allocator() *memory.Allocator {
return ec.es.alloc
}
func (ec executionContext) Parents() []DatasetID {
return ec.parents
}
func (ec executionContext) Dependencies() Dependencies {
return ec.es.deps
}