/
bigmachine.go
1580 lines (1468 loc) · 46.6 KB
/
bigmachine.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
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2018 GRAIL, Inc. All rights reserved.
// Use of this source code is governed by the Apache 2.0
// license that can be found in the LICENSE file.
package exec
import (
"bufio"
"context"
"encoding/gob"
"fmt"
"io"
"io/ioutil"
"math/rand"
"net/http"
"runtime"
"runtime/debug"
"sync"
"time"
"github.com/grailbio/base/backgroundcontext"
"github.com/grailbio/base/errors"
"github.com/grailbio/base/eventlog"
"github.com/grailbio/base/limitbuf"
"github.com/grailbio/base/limiter"
"github.com/grailbio/base/log"
"github.com/grailbio/base/retry"
"github.com/grailbio/base/status"
"github.com/grailbio/base/sync/ctxsync"
"github.com/grailbio/base/sync/once"
"github.com/grailbio/bigmachine"
"github.com/grailbio/bigslice"
"github.com/grailbio/bigslice/frame"
"github.com/grailbio/bigslice/metrics"
"github.com/grailbio/bigslice/sliceio"
"github.com/grailbio/bigslice/stats"
"golang.org/x/sync/errgroup"
)
const BigmachineStatusGroup = "bigmachine"
func init() {
gob.Register(invocationRef{})
}
const (
// StatsPollInterval is the period at which task statistics are polled.
statsPollInterval = 10 * time.Second
// StatTimeout is the maximum amount of time allowed to retrieve
// machine stats, per iteration.
statTimeout = 5 * time.Second
)
// RetryPolicy is the default retry policy used for machine calls.
var retryPolicy = retry.MaxRetries(retry.Backoff(5*time.Second, 60*time.Second, 2), 5)
// FatalErr is used to match fatal errors.
var fatalErr = errors.E(errors.Fatal)
// DoShuffleReaders determines whether reader tasks should be
// shuffled in order to avoid potential thundering herd issues.
// This should only be used in testing when deterministic ordering
// matters.
//
// TODO(marius): make this a session option instead.
var DoShuffleReaders = true
func init() {
gob.Register(&worker{})
}
// TODO(marius): clean up flag registration, etc. vis-a-vis bigmachine.
// e.g., perhaps we can register flags in a bigmachine flagset that gets
// parsed together, so that we don't litter the process with global flags.
// BigmachineExecutor is an executor that runs individual tasks on
// bigmachine machines.
type bigmachineExecutor struct {
system bigmachine.System
params []bigmachine.Param
sess *Session
b *bigmachine.B
status *status.Group
mu sync.Mutex
locations map[*Task]*sliceMachine
stats map[string]stats.Values
// Invocations and invocationDeps are used to track dependencies
// between invocations so that we can execute arbitrary graphs of
// slices on bigmachine workers. Note that this requires that we
// hold on to the invocations, which is somewhat unfortunate, but
// I don't see a clean way around it.
invocations map[uint64]execInvocation
invocationDeps map[uint64]map[uint64]bool
// encodedInvocations holds the gob-encoded representations of the
// corresponding invocations. Because Func arguments, held in invocations,
// may be large, we memoize the encoded versions so that we don't pay the
// cost of gob-encoding the invocations for each worker (CPU to encode;
// memory for ephemeral buffers in gob). Instead, we do it once and reuse
// the result for each worker.
encodedInvocations *invDiskCache
// Worker is the (configured) worker service to instantiate on
// allocated machines.
worker *worker
// Managers is the set of machine machine managers used by this
// executor. Even managers use the session's maxload, and will
// share task load on a single machine. Odd managers are used
// for exclusive tasks.
//
// Thus manager selection proceeds as follows: the default manager
// is managers[0]. Func-exclusive tasks use managers[invocation].
managers []*machineManager
}
func newBigmachineExecutor(system bigmachine.System, params ...bigmachine.Param) *bigmachineExecutor {
return &bigmachineExecutor{system: system, params: params}
}
func (b *bigmachineExecutor) Name() string {
return "bigmachine:" + b.system.Name()
}
// Start starts registers the bigslice worker with bigmachine and then
// starts the bigmachine.
//
// TODO(marius): provide fine-grained fault tolerance.
func (b *bigmachineExecutor) Start(sess *Session) (shutdown func()) {
b.sess = sess
b.b = bigmachine.Start(b.system)
b.locations = make(map[*Task]*sliceMachine)
b.stats = make(map[string]stats.Values)
if status := sess.Status(); status != nil {
b.status = status.Group(BigmachineStatusGroup)
}
b.invocations = make(map[uint64]execInvocation)
b.invocationDeps = make(map[uint64]map[uint64]bool)
b.encodedInvocations = newInvDiskCache()
b.worker = &worker{
MachineCombiners: sess.machineCombiners,
}
return func() {
b.b.Shutdown()
b.encodedInvocations.close()
}
}
func (b *bigmachineExecutor) manager(i int) *machineManager {
b.mu.Lock()
defer b.mu.Unlock()
for i >= len(b.managers) {
b.managers = append(b.managers, nil)
}
if b.managers[i] == nil {
maxLoad := b.sess.MaxLoad()
b.managers[i] = newMachineManager(b.b, b.params, b.status, b.sess.Parallelism(), maxLoad, b.worker)
go b.managers[i].Do(backgroundcontext.Get())
}
return b.managers[i]
}
// checkInvocationReader checks that we can successfully read out the
// serialized version of invocation invIndex in b's graph, as we will do when
// we transport it to workers. As a side effect, the serialized invocation is
// cached for future use.
//
// The invocation with index invIndex must previously have been added with
// addInvocation.
func (b *bigmachineExecutor) checkInvocationReader(invIndex uint64) (err error) {
rc, err := b.invocationReader(invIndex)
if err != nil {
return err
}
defer errors.CleanUp(rc.Close, &err)
if _, err = io.Copy(io.Discard, rc); err != nil {
return err
}
return nil
}
// invocationReader returns a reader that reads the serialized version of
// invocation invIndex in b's graph for transport to workers.
//
// The invocation with index invIndex must previously have been added with
// addInvocation.
func (b *bigmachineExecutor) invocationReader(invIndex uint64) (io.ReadCloser, error) {
b.mu.Lock()
inv := b.invocations[invIndex]
b.mu.Unlock()
return b.encodedInvocations.getOrCreate(inv.Index, func(w io.Writer) error {
if err := gob.NewEncoder(w).Encode(inv); err != nil {
return errors.E(errors.Fatal, errors.Invalid, "gob-encoding invocation", err)
}
return nil
})
}
// addInvocation adds an invocation to b's invocation graph. Returns true iff
// the invocation was added for the first time.
func (b *bigmachineExecutor) addInvocation(inv execInvocation) (bool, error) {
b.mu.Lock()
defer b.mu.Unlock()
if _, ok := b.invocations[inv.Index]; ok {
return false, nil
}
// This is the first time we are seeing this invocation.
// Each *Result argument represents a dependency on other invocations.
// Substitute each *Result argument for an invocationRef so that the
// result/dependency may be transported to worker machines. See
// (*worker).Compile.
for i, arg := range inv.Args {
result, ok := arg.(*Result)
if !ok {
continue
}
if _, ok := b.invocations[result.invIndex]; !ok {
panic(fmt.Sprintf("result from unknown invocation %d", result.invIndex))
}
inv.Args[i] = invocationRef{result.invIndex}
if b.invocationDeps[inv.Index] == nil {
b.invocationDeps[inv.Index] = make(map[uint64]bool)
}
b.invocationDeps[inv.Index][result.invIndex] = true
}
b.invocations[inv.Index] = inv
return true, nil
}
func (b *bigmachineExecutor) compile(ctx context.Context, m *sliceMachine, invIndex uint64) error {
b.mu.Lock()
// Traverse the invocation graph bottom-up, making sure everything on the
// machine is compiled. We produce a valid order, but we don't capture
// opportunities for parallel compilations.
// TODO(marius): allow for parallel compilation as some users are
// performing expensive computations inside of bigslice.Funcs.
var (
todo = []uint64{invIndex}
invocations []execInvocation
)
for len(todo) > 0 {
var i uint64
i, todo = todo[0], todo[1:]
invocations = append(invocations, b.invocations[i])
for j := range b.invocationDeps[i] {
todo = append(todo, j)
}
}
b.mu.Unlock()
for i := len(invocations) - 1; i >= 0; i-- {
err := m.Compiles.Do(invocations[i].Index, func() error {
inv := invocations[i]
// Flatten these into lists so that we don't capture further
// structure by JSON encoding down the line. We also truncate them
// so that, e.g., huge lists of arguments don't make it into the trace.
args := make([]string, len(inv.Args))
for i := range args {
args[i] = truncatef(inv.Args[i])
}
b.sess.tracer.Event(m, inv, "B", "location", inv.Location, "args", args)
makeInvReader := func() (io.Reader, error) {
return b.invocationReader(inv.Index)
}
err := m.RetryCall(ctx, "Worker.Compile", makeInvReader, nil)
if err != nil {
b.sess.tracer.Event(m, inv, "E", "error", err)
} else {
b.sess.tracer.Event(m, inv, "E")
}
return err
})
if err != nil {
return err
}
}
return nil
}
func (b *bigmachineExecutor) commit(ctx context.Context, m *sliceMachine, key string) error {
return m.Commits.Do(key, func() error {
return m.RetryCall(ctx, "Worker.CommitCombiner", TaskName{Op: key}, nil)
})
}
func (b *bigmachineExecutor) Run(task *Task) {
task.Status.Print("waiting for a machine")
invIndex := task.Invocation.Index
added, err := b.addInvocation(task.Invocation)
if err != nil {
task.Errorf("error adding invocation %d to graph: %v", invIndex, err)
return
}
if added {
// This is the first time we are seeing this invocation. Check if we
// can serialize and cache the invocation that we will ultimately send
// to worker machines. Do this eagerly to discover errors quickly,
// rather than potentially waiting until machines are started.
if err := b.checkInvocationReader(invIndex); err != nil {
task.Errorf("error serializing invocation %d: %v", invIndex, err)
return
}
}
// Use the default/shared cluster unless the func is exclusive.
var cluster int
if task.Invocation.Exclusive {
cluster = int(invIndex)
}
mgr := b.manager(cluster)
procs := task.Pragma.Procs()
if task.Pragma.Exclusive() || procs > mgr.machprocs {
procs = mgr.machprocs
}
var (
ctx = backgroundcontext.Get()
offerc, cancel = mgr.Offer(int(invIndex), procs)
m *sliceMachine
)
select {
case <-ctx.Done():
task.Error(ctx.Err())
cancel()
return
case m = <-offerc:
}
numTasks := m.Stats.Int("tasks")
numTasks.Add(1)
m.UpdateStatus()
defer func() {
numTasks.Add(-1)
m.UpdateStatus()
}()
// Make sure that the invocation has been compiled on the selected
// machine.
compile:
for {
err := b.compile(ctx, m, invIndex)
switch {
case err == nil:
break compile
case ctx.Err() == nil && (err == context.Canceled || err == context.DeadlineExceeded):
// In this case, we've caught a context error from a prior
// invocation. We're going to try to run it again. Note that this
// is racy: the behavior remains correct but may imply additional
// data transfer. C'est la vie.
m.Compiles.Forget(invIndex)
case errors.Is(errors.Invalid, err) && errors.Match(fatalErr, err):
// Fatally invalid compilation parameters, e.g. func arguments that
// are not gob-encodable, are fatal to the task.
fallthrough
case errors.Is(errors.Remote, err):
// Compilations don't involve invoking user code, nor do they
// involve dependencies other than potentially uploading data from
// the driver node, so we consider any error to be fatal to the task.
task.Errorf("failed to compile invocation on machine %s: %v", m.Addr, err)
m.Done(procs, err)
return
default:
task.Status.Printf("task lost while compiling bigslice.Func: %v", err)
task.Set(TaskLost)
m.Done(procs, err)
return
}
}
// Populate the run request. Include the locations of all dependent
// outputs so that the receiving worker can read from them.
req := taskRunRequest{
Name: task.Name,
Invocation: invIndex,
}
machineIndices := make(map[string]int)
g, _ := errgroup.WithContext(ctx)
for _, dep := range task.Deps {
for i := 0; i < dep.NumTask(); i++ {
deptask := dep.Task(i)
depm := b.location(deptask)
if depm == nil {
// TODO(marius): make this a separate state, or a separate
// error type?
task.Errorf("task %v has no location", deptask)
m.Done(procs, nil)
return
}
j, ok := machineIndices[depm.Addr]
if !ok {
j = len(machineIndices)
machineIndices[depm.Addr] = j
req.Machines = append(req.Machines, depm.Addr)
}
req.Locations = append(req.Locations, j)
key := dep.CombineKey
if key == "" {
continue
}
// Make sure that the result is committed.
g.Go(func() error { return b.commit(ctx, depm, key) })
}
}
task.Status.Print(m.Addr)
if err := g.Wait(); err != nil {
task.Errorf("failed to commit combiner: %v", err)
return
}
// While we're running, also update task stats directly into the tasks's status.
// TODO(marius): also aggregate stats across all tasks.
statsCtx, statsCancel := context.WithCancel(ctx)
go monitorTaskStats(statsCtx, m, task)
b.sess.tracer.Event(m, task, "B")
task.Set(TaskRunning)
var reply taskRunReply
err = m.RetryCall(ctx, "Worker.Run", req, &reply)
statsCancel()
m.Done(procs, err)
switch {
case err == nil:
// Convert nanoseconds to microseconds to be same units as event durations.
b.sess.tracer.Event(m, task, "E",
"readDuration", reply.Vals["readDuration"]/1e3,
"writeDuration", reply.Vals["writeDuration"]/1e3,
)
b.setLocation(task, m)
task.Status.Printf("done: %s", reply.Vals)
task.Scope.Reset(&reply.Scope)
task.Set(TaskOk)
m.Assign(task)
case ctx.Err() != nil:
b.sess.tracer.Event(m, task, "E", "error", ctx.Err())
task.Error(err)
case errors.Is(errors.Remote, err) && errors.Match(fatalErr, err):
b.sess.tracer.Event(m, task, "E", "error", err, "error_type", "fatal")
// Fatal errors aren't retryable.
task.Error(err)
default:
// Everything else we consider as the task being lost. It'll get
// resubmitted by the evaluator.
b.sess.tracer.Event(m, task, "E", "error", err, "error_type", "lost")
task.Status.Printf("lost task during task evaluation: %v", err)
task.Set(TaskLost)
}
}
// monitorTaskStats monitors stats (e.g. records read/written) of the task
// running on m, updating task's status until ctx is done.
func monitorTaskStats(ctx context.Context, m *sliceMachine, task *Task) {
wait := func() {
select {
case <-time.After(statsPollInterval):
case <-ctx.Done():
}
}
for ctx.Err() == nil {
var vals *stats.Values
err := m.RetryCall(ctx, "Worker.TaskStats", task.Name, &vals)
if err != nil {
log.Error.Printf("error getting task stats from %s: %v", m.Addr, err)
wait()
continue
}
task.Status.Printf("%s: %s", m.Addr, *vals)
wait()
}
}
func (b *bigmachineExecutor) Reader(task *Task, partition int) sliceio.ReadCloser {
m := b.location(task)
if m == nil {
return sliceio.NopCloser(sliceio.ErrReader(errors.E(errors.NotExist, fmt.Sprintf("task %s", task.Name))))
}
if task.CombineKey != "" {
return sliceio.NopCloser(sliceio.ErrReader(fmt.Errorf("read %s: cannot read tasks with combine keys", task.Name)))
}
// TODO(marius): access the store here, too, in case it's a shared one (e.g., s3)
return newEvalReader(b, task, partition)
}
func (b *bigmachineExecutor) Discard(ctx context.Context, task *Task) {
if !task.Combiner.IsNil() && task.CombineKey != "" {
// We do not yet handle tasks with shared combiners.
return
}
task.Lock()
if task.state != TaskOk {
// We have no results to discard if the task is not TaskOk, as it has
// not completed successfully.
task.Unlock()
return
}
task.state = TaskRunning
task.Unlock()
m := b.location(task)
if m == nil {
return
}
m.Discard(ctx, task)
}
func (b *bigmachineExecutor) Eventer() eventlog.Eventer {
return b.sess.eventer
}
func (b *bigmachineExecutor) HandleDebug(handler *http.ServeMux) {
b.b.HandleDebug(handler)
}
// Location returns the machine on which the results of the provided
// task resides.
func (b *bigmachineExecutor) location(task *Task) *sliceMachine {
b.mu.Lock()
m := b.locations[task]
b.mu.Unlock()
return m
}
func (b *bigmachineExecutor) setLocation(task *Task, m *sliceMachine) {
b.mu.Lock()
b.locations[task] = m
b.mu.Unlock()
}
type combinerState int
const (
combinerNone combinerState = iota
combinerWriting
combinerCommitted
combinerError
combinerIdle
// States > combinerIdle are reference counts.
)
// A worker is the bigmachine service that runs individual tasks and serves
// the results of previous runs. Currently all output is buffered in memory.
type worker struct {
// MachineCombiners determines whether to use the MachineCombiners
// compilation option.
MachineCombiners bool
b *bigmachine.B
store Store
mu sync.Mutex
cond *ctxsync.Cond
compiles once.Map
tasks map[uint64]map[TaskName]*Task
taskStats map[uint64]map[TaskName]*stats.Map
slices map[uint64]bigslice.Slice
stats *stats.Map
// CombinerStates and combiners are used to manage shared combine
// buffers. combinerErrors is used to track the original cause of an
// a combiner error and report it accordingly.
combinerStates map[TaskName]combinerState
combinerErrors map[TaskName]error
combiners map[TaskName][]chan *combiner
commitLimiter *limiter.Limiter
}
func (w *worker) Init(b *bigmachine.B) error {
w.cond = ctxsync.NewCond(&w.mu)
w.tasks = make(map[uint64]map[TaskName]*Task)
w.taskStats = make(map[uint64]map[TaskName]*stats.Map)
w.slices = make(map[uint64]bigslice.Slice)
w.combiners = make(map[TaskName][]chan *combiner)
w.combinerStates = make(map[TaskName]combinerState)
w.combinerErrors = make(map[TaskName]error)
w.b = b
dir, err := ioutil.TempDir("", "bigslice")
if err != nil {
return err
}
w.store = &fileStore{Prefix: dir + "/"}
w.stats = stats.NewMap()
// Set up a limiter to limit the number of concurrent commits
// that are allowed to happen in the worker.
//
// TODO(marius): we should treat commits like tasks and apply
// load balancing/limiting instead.
w.commitLimiter = limiter.New()
procs := b.System().Maxprocs()
if procs == 0 {
procs = runtime.GOMAXPROCS(0)
}
w.commitLimiter.Release(procs)
return nil
}
// FuncLocations produces a slice of strings that describe the locations of
// Func creation. It is used to verify that this process is working from an
// identical Func registry. See bigslice.FuncLocations for more information.
func (w *worker) FuncLocations(ctx context.Context, _ struct{}, locs *[]string) error {
*locs = bigslice.FuncLocations()
return nil
}
// Compile compiles an invocation on the worker and stores the
// resulting tasks. Compile is idempotent: it will compile each
// invocation at most once.
func (w *worker) Compile(ctx context.Context, invReader io.Reader, _ *struct{}) (err error) {
defer func() {
if e := recover(); e != nil {
err = fmt.Errorf("invocation panic! %v", e)
err = errors.E(errors.Fatal, err)
}
}()
var inv execInvocation
if err = gob.NewDecoder(invReader).Decode(&inv); err != nil {
return errors.E(errors.Invalid, "error gob-decoding invocation", err)
}
return w.compiles.Do(inv.Index, func() error {
// Substitute invocation refs for the results of the invocation.
// The executor must ensure that all references have been compiled.
for i, arg := range inv.Args {
ref, ok := arg.(invocationRef)
if !ok {
continue
}
w.mu.Lock()
inv.Args[i], ok = w.slices[ref.Index]
w.mu.Unlock()
if !ok {
return fmt.Errorf("worker.Compile: invalid invocation reference %x", ref.Index)
}
}
slice := inv.Invoke()
tasks, err := compile(inv, slice, w.MachineCombiners)
if err != nil {
return err
}
all := make(map[*Task]bool)
for _, task := range tasks {
task.all(all)
}
named := make(map[TaskName]*Task)
for task := range all {
named[task.Name] = task
}
namedStats := make(map[TaskName]*stats.Map)
for task := range all {
namedStats[task.Name] = stats.NewMap()
}
w.mu.Lock()
w.tasks[inv.Index] = named
w.taskStats[inv.Index] = namedStats
w.slices[inv.Index] = &Result{Slice: slice, tasks: tasks}
w.mu.Unlock()
return nil
})
}
// TaskRunRequest contains all data required to run an individual task.
type taskRunRequest struct {
// Invocation is the invocation from which the task was compiled.
Invocation uint64
// Name is the name of the task compiled from Invocation.
Name TaskName
// Machines stores the set of machines indexed in Locations.
Machines []string
// Locations indexes machine locations for task outputs. Locations
// stores the machine index of each task dependency. We rely on the
// fact that the task graph is identical to all viewers: locations
// are stored in the order of task dependencies.
Locations []int
}
func (r *taskRunRequest) location(taskIndex int) string {
return r.Machines[r.Locations[taskIndex]]
}
type taskRunReply struct {
// Vals are the stat values for the run of the task.
Vals stats.Values
// Scope is the scope of the task at completion time.
// TODO(marius): unify scopes with values, above.
Scope metrics.Scope
}
// maybeTaskFatalErr wraps errors in (*worker).Run that can cause fatal task
// errors, errors that will cause the evaluator to mark the task in TaskErr
// state and halt evaluation. This is generally used to identify (fatal) errors
// returned by application code. These errors are not returned from
// (*worker).Run; they are used internally to revise severity.
type maybeTaskFatalErr struct {
error
}
// reviseSeverity revises the severity of err for (*worker).Run. (*worker).Run
// only returns fatal errors for task fatal errors, errors that will cause tasks
// on the driver to be marked TaskErr and halt evaluation.
func reviseSeverity(err error) error {
if err == nil {
return nil
}
if e, ok := err.(maybeTaskFatalErr); ok {
return e.error
}
if e, ok := err.(*errors.Error); ok && e != nil && e.Severity == errors.Fatal {
// The error is fatal to this attempt to run the task but not fatal to
// the task overall, e.g. a fatal unavailable error when trying to read
// dependencies from other machines. We downgrade the error, so that the
// evaluator will retry.
e.Severity = errors.Unknown
return e
}
return err
}
// Run runs an individual task as described in the request. Run returns a nil
// error when the task was successfully run and its output deposited in a local
// buffer. If Run returns a *errors.Error with errors.Fatal severity, the task
// will be marked in TaskErr, and evaluation will halt.
func (w *worker) Run(ctx context.Context, req taskRunRequest, reply *taskRunReply) (err error) {
var task *Task
defer func() {
if e := recover(); e != nil {
stack := debug.Stack()
err = fmt.Errorf("panic while evaluating slice: %v\n%s", e, string(stack))
err = maybeTaskFatalErr{errors.E(err, errors.Fatal)}
}
if err != nil {
log.Error.Printf("task %s error: %v", req.Name, err)
err = reviseSeverity(err)
if task != nil {
task.Error(errors.Recover(err))
}
return
}
if task != nil {
task.Set(TaskOk)
}
}()
w.mu.Lock()
named := w.tasks[req.Invocation]
namedStats := w.taskStats[req.Invocation]
w.mu.Unlock()
if named == nil {
return maybeTaskFatalErr{errors.E(errors.Fatal, fmt.Errorf("invocation %x not compiled", req.Invocation))}
}
task = named[req.Name]
if task == nil {
return maybeTaskFatalErr{errors.E(errors.Fatal, fmt.Errorf("task %s not found", req.Name))}
}
taskStats := namedStats[req.Name]
ctx = metrics.ScopedContext(ctx, &task.Scope)
defer func() {
reply.Vals = make(stats.Values)
taskStats.AddAll(reply.Vals)
reply.Scope.Reset(&task.Scope)
}()
task.Lock()
switch task.state {
case TaskLost:
log.Printf("Worker.Run: %s: reviving LOST task", task.Name)
case TaskErr:
log.Printf("Worker.Run: %s: reviving FAILED task", task.Name)
case TaskInit:
default:
for task.state <= TaskRunning {
log.Printf("runtask: %s already running. Waiting for it to finish.", task.Name)
err = task.Wait(ctx)
if err != nil {
break
}
}
task.Unlock()
if e := task.Err(); e != nil {
err = e
}
return err
}
task.state = TaskRunning
task.Unlock()
// Gather inputs from the bigmachine cluster, dialing machines
// as necessary.
var (
// Stats for the task.
taskTotalRecordsIn *stats.Int
taskRecordsIn *stats.Int
taskRecordsOut = taskStats.Int("write")
taskReadDuration = taskStats.Int("readDuration")
taskWriteDuration = taskStats.Int("writeDuration")
// Stats for the machine.
totalRecordsIn *stats.Int
recordsIn *stats.Int
recordsOut = w.stats.Int("write")
)
taskRecordsOut.Set(0)
if len(task.Deps) > 0 {
taskTotalRecordsIn = taskStats.Int("inrecords")
taskTotalRecordsIn.Set(0)
taskRecordsIn = taskStats.Int("read")
taskRecordsIn.Set(0)
totalRecordsIn = w.stats.Int("inrecords")
recordsIn = w.stats.Int("read")
}
var (
in = make([]sliceio.Reader, 0, len(task.Deps))
taskIndex int
)
for _, dep := range task.Deps {
// If the dependency has a combine key, they are combined on the
// machine, and we de-dup the dependencies.
//
// The caller of has already ensured that the combiner buffers
// are committed on the machines.
if dep.CombineKey != "" {
locations := make(map[string]bool)
for i := 0; i < dep.NumTask(); i++ {
addr := req.location(taskIndex)
taskIndex++
// We only read the first combine key for each location.
//
// TODO(marius): compute some non-overlapping intersection of
// combine keys instead, so that we can handle error recovery
// properly. In particular, in the case of error recovery, we
// have to create new combiner keys so that they aren't written
// into previous combiner buffers. This suggests that combiner
// keys should be assigned by the executor, and not during
// compile time.
if locations[addr] {
continue
}
locations[addr] = true
}
for addr := range locations {
machine, err := w.b.Dial(ctx, addr)
if err != nil {
return err
}
r := newMachineReader(machine, taskPartition{TaskName{Op: dep.CombineKey}, dep.Partition})
in = append(in, &statsReader{r, []*stats.Int{taskRecordsIn, recordsIn}, taskReadDuration})
defer r.Close()
}
} else {
reader := new(multiReader)
reader.q = make([]sliceio.Reader, dep.NumTask())
Tasks:
for j := 0; j < dep.NumTask(); j++ {
deptask := dep.Task(j)
// If we have it locally, or if we're using a shared backend store
// (e.g., S3), then read it directly.
info, err := w.store.Stat(ctx, deptask.Name, dep.Partition)
if err == nil {
rc, openErr := w.store.Open(ctx, deptask.Name, dep.Partition, 0)
if openErr == nil {
defer rc.Close()
r := sliceio.NewDecodingReader(rc)
reader.q[j] = &statsReader{r, []*stats.Int{taskRecordsIn, recordsIn}, taskReadDuration}
taskTotalRecordsIn.Add(info.Records)
totalRecordsIn.Add(info.Records)
taskIndex++
continue Tasks
}
}
// Find the location of the task.
addr := req.location(taskIndex)
taskIndex++
machine, err := w.b.Dial(ctx, addr)
if err != nil {
return err
}
tp := taskPartition{deptask.Name, dep.Partition}
if err := machine.RetryCall(ctx, "Worker.Stat", tp, &info); err != nil {
return err
}
r := newMachineReader(machine, tp)
reader.q[j] = &statsReader{r, []*stats.Int{taskRecordsIn, recordsIn}, taskReadDuration}
taskTotalRecordsIn.Add(info.Records)
totalRecordsIn.Add(info.Records)
defer r.Close()
}
// We shuffle the tasks here so that we don't encounter
// "thundering herd" issues were partitions are read sequentially
// from the same (ordered) list of machines.
//
// TODO(marius): possibly we should perform proper load balancing
// here
if DoShuffleReaders {
rand.Shuffle(len(reader.q), func(i, j int) { reader.q[i], reader.q[j] = reader.q[j], reader.q[i] })
}
if dep.Expand {
in = append(in, reader.q...)
} else {
in = append(in, reader)
}
}
}
// If we have a combiner, then we partition globally for the machine
// into common combiners.
if !task.Combiner.IsNil() {
return w.runCombine(ctx, task, taskStats, task.Do(in))
}
// Stream partition output directly to the underlying store, but
// through a buffer because the column encoder can make small
// writes.
//
// TODO(marius): switch to using a monotasks-like arrangement
// instead once we also have memory management, in order to control
// buffer growth.
type partition struct {
wc writeCommitter
buf *bufio.Writer
sliceio.Writer
}
partitions := make([]*partition, task.NumPartition)
for p := range partitions {
wc, err := w.store.Create(ctx, task.Name, p)
if err != nil {
return err
}
// TODO(marius): pool the writers so we can reuse them.
part := new(partition)
part.wc = wc
part.buf = bufio.NewWriter(wc)
part.Writer = &statsWriter{sliceio.NewEncodingWriter(part.buf), taskWriteDuration}
partitions[p] = part
}
defer func() {
for _, part := range partitions {
if part == nil {
continue
}
part.wc.Discard(ctx)
}
}()
out := task.Do(in)
count := make([]int64, task.NumPartition)
switch {
case task.NumOut() == 0:
// If there are no output columns, just drive the computation.
_, err := out.Read(ctx, frame.Empty)
if err != nil && err != sliceio.EOF {
return maybeTaskFatalErr{err}
}
return nil
case task.NumPartition > 1:
var psize = (*defaultChunksize + 99) / 100
var (
partitionv = make([]frame.Frame, task.NumPartition)
lens = make([]int, task.NumPartition)
shards = make([]int, *defaultChunksize)
)
for i := range partitionv {
partitionv[i] = frame.Make(task, psize, psize)
}
in := frame.Make(task, *defaultChunksize, *defaultChunksize)
for {
n, err := out.Read(ctx, in)
if err != nil && err != sliceio.EOF {
return maybeTaskFatalErr{err}
}
task.Partitioner(ctx, in, task.NumPartition, shards[:n])
for i := 0; i < n; i++ {
p := shards[i]
j := lens[p]
frame.Copy(partitionv[p].Slice(j, j+1), in.Slice(i, i+1))
lens[p]++
count[p]++
// Flush when we fill up.
if lens[p] == psize {
if writeErr := partitions[p].Write(ctx, partitionv[p]); writeErr != nil {
return maybeTaskFatalErr{errors.E(errors.Fatal, writeErr)}
}
lens[p] = 0
}
}
taskRecordsOut.Add(int64(n))
recordsOut.Add(int64(n))
if err == sliceio.EOF {
break
}
}
// Flush remaining data.
for p, n := range lens {
if n == 0 {
continue