github.com/grailbio/bigslice@v0.0.0-20230519005545-30c4c12152ad/exec/bigmachine.go (about) 1 // Copyright 2018 GRAIL, Inc. All rights reserved. 2 // Use of this source code is governed by the Apache 2.0 3 // license that can be found in the LICENSE file. 4 5 package exec 6 7 import ( 8 "bufio" 9 "context" 10 "encoding/gob" 11 "fmt" 12 "io" 13 "io/ioutil" 14 "math/rand" 15 "net/http" 16 "runtime" 17 "runtime/debug" 18 "sync" 19 "time" 20 21 "github.com/grailbio/base/backgroundcontext" 22 "github.com/grailbio/base/errors" 23 "github.com/grailbio/base/eventlog" 24 "github.com/grailbio/base/limitbuf" 25 "github.com/grailbio/base/limiter" 26 "github.com/grailbio/base/log" 27 "github.com/grailbio/base/retry" 28 "github.com/grailbio/base/status" 29 "github.com/grailbio/base/sync/ctxsync" 30 "github.com/grailbio/base/sync/once" 31 "github.com/grailbio/bigmachine" 32 "github.com/grailbio/bigslice" 33 "github.com/grailbio/bigslice/frame" 34 "github.com/grailbio/bigslice/metrics" 35 "github.com/grailbio/bigslice/sliceio" 36 "github.com/grailbio/bigslice/stats" 37 "golang.org/x/sync/errgroup" 38 ) 39 40 const BigmachineStatusGroup = "bigmachine" 41 42 func init() { 43 gob.Register(invocationRef{}) 44 } 45 46 const ( 47 // StatsPollInterval is the period at which task statistics are polled. 48 statsPollInterval = 10 * time.Second 49 50 // StatTimeout is the maximum amount of time allowed to retrieve 51 // machine stats, per iteration. 52 statTimeout = 5 * time.Second 53 ) 54 55 // RetryPolicy is the default retry policy used for machine calls. 56 var retryPolicy = retry.MaxRetries(retry.Backoff(5*time.Second, 60*time.Second, 2), 5) 57 58 // FatalErr is used to match fatal errors. 59 var fatalErr = errors.E(errors.Fatal) 60 61 // DoShuffleReaders determines whether reader tasks should be 62 // shuffled in order to avoid potential thundering herd issues. 63 // This should only be used in testing when deterministic ordering 64 // matters. 65 // 66 // TODO(marius): make this a session option instead. 67 var DoShuffleReaders = true 68 69 func init() { 70 gob.Register(&worker{}) 71 } 72 73 // TODO(marius): clean up flag registration, etc. vis-a-vis bigmachine. 74 // e.g., perhaps we can register flags in a bigmachine flagset that gets 75 // parsed together, so that we don't litter the process with global flags. 76 77 // BigmachineExecutor is an executor that runs individual tasks on 78 // bigmachine machines. 79 type bigmachineExecutor struct { 80 system bigmachine.System 81 params []bigmachine.Param 82 83 sess *Session 84 b *bigmachine.B 85 86 status *status.Group 87 88 mu sync.Mutex 89 90 locations map[*Task]*sliceMachine 91 stats map[string]stats.Values 92 93 // Invocations and invocationDeps are used to track dependencies 94 // between invocations so that we can execute arbitrary graphs of 95 // slices on bigmachine workers. Note that this requires that we 96 // hold on to the invocations, which is somewhat unfortunate, but 97 // I don't see a clean way around it. 98 invocations map[uint64]execInvocation 99 invocationDeps map[uint64]map[uint64]bool 100 101 // encodedInvocations holds the gob-encoded representations of the 102 // corresponding invocations. Because Func arguments, held in invocations, 103 // may be large, we memoize the encoded versions so that we don't pay the 104 // cost of gob-encoding the invocations for each worker (CPU to encode; 105 // memory for ephemeral buffers in gob). Instead, we do it once and reuse 106 // the result for each worker. 107 encodedInvocations *invDiskCache 108 109 // Worker is the (configured) worker service to instantiate on 110 // allocated machines. 111 worker *worker 112 113 // Managers is the set of machine machine managers used by this 114 // executor. Even managers use the session's maxload, and will 115 // share task load on a single machine. Odd managers are used 116 // for exclusive tasks. 117 // 118 // Thus manager selection proceeds as follows: the default manager 119 // is managers[0]. Func-exclusive tasks use managers[invocation]. 120 managers []*machineManager 121 } 122 123 func newBigmachineExecutor(system bigmachine.System, params ...bigmachine.Param) *bigmachineExecutor { 124 return &bigmachineExecutor{system: system, params: params} 125 } 126 127 func (b *bigmachineExecutor) Name() string { 128 return "bigmachine:" + b.system.Name() 129 } 130 131 // Start starts registers the bigslice worker with bigmachine and then 132 // starts the bigmachine. 133 // 134 // TODO(marius): provide fine-grained fault tolerance. 135 func (b *bigmachineExecutor) Start(sess *Session) (shutdown func()) { 136 b.sess = sess 137 b.b = bigmachine.Start(b.system) 138 b.locations = make(map[*Task]*sliceMachine) 139 b.stats = make(map[string]stats.Values) 140 if status := sess.Status(); status != nil { 141 b.status = status.Group(BigmachineStatusGroup) 142 } 143 b.invocations = make(map[uint64]execInvocation) 144 b.invocationDeps = make(map[uint64]map[uint64]bool) 145 b.encodedInvocations = newInvDiskCache() 146 b.worker = &worker{ 147 MachineCombiners: sess.machineCombiners, 148 } 149 150 return func() { 151 b.b.Shutdown() 152 b.encodedInvocations.close() 153 } 154 } 155 156 func (b *bigmachineExecutor) manager(i int) *machineManager { 157 b.mu.Lock() 158 defer b.mu.Unlock() 159 for i >= len(b.managers) { 160 b.managers = append(b.managers, nil) 161 } 162 if b.managers[i] == nil { 163 maxLoad := b.sess.MaxLoad() 164 b.managers[i] = newMachineManager(b.b, b.params, b.status, b.sess.Parallelism(), maxLoad, b.worker) 165 go b.managers[i].Do(backgroundcontext.Get()) 166 } 167 return b.managers[i] 168 } 169 170 // checkInvocationReader checks that we can successfully read out the 171 // serialized version of invocation invIndex in b's graph, as we will do when 172 // we transport it to workers. As a side effect, the serialized invocation is 173 // cached for future use. 174 // 175 // The invocation with index invIndex must previously have been added with 176 // addInvocation. 177 func (b *bigmachineExecutor) checkInvocationReader(invIndex uint64) (err error) { 178 rc, err := b.invocationReader(invIndex) 179 if err != nil { 180 return err 181 } 182 defer errors.CleanUp(rc.Close, &err) 183 if _, err = io.Copy(io.Discard, rc); err != nil { 184 return err 185 } 186 return nil 187 } 188 189 // invocationReader returns a reader that reads the serialized version of 190 // invocation invIndex in b's graph for transport to workers. 191 // 192 // The invocation with index invIndex must previously have been added with 193 // addInvocation. 194 func (b *bigmachineExecutor) invocationReader(invIndex uint64) (io.ReadCloser, error) { 195 b.mu.Lock() 196 inv := b.invocations[invIndex] 197 b.mu.Unlock() 198 return b.encodedInvocations.getOrCreate(inv.Index, func(w io.Writer) error { 199 if err := gob.NewEncoder(w).Encode(inv); err != nil { 200 return errors.E(errors.Fatal, errors.Invalid, "gob-encoding invocation", err) 201 } 202 return nil 203 }) 204 } 205 206 // addInvocation adds an invocation to b's invocation graph. Returns true iff 207 // the invocation was added for the first time. 208 func (b *bigmachineExecutor) addInvocation(inv execInvocation) (bool, error) { 209 b.mu.Lock() 210 defer b.mu.Unlock() 211 if _, ok := b.invocations[inv.Index]; ok { 212 return false, nil 213 } 214 // This is the first time we are seeing this invocation. 215 216 // Each *Result argument represents a dependency on other invocations. 217 // Substitute each *Result argument for an invocationRef so that the 218 // result/dependency may be transported to worker machines. See 219 // (*worker).Compile. 220 for i, arg := range inv.Args { 221 result, ok := arg.(*Result) 222 if !ok { 223 continue 224 } 225 if _, ok := b.invocations[result.invIndex]; !ok { 226 panic(fmt.Sprintf("result from unknown invocation %d", result.invIndex)) 227 } 228 inv.Args[i] = invocationRef{result.invIndex} 229 if b.invocationDeps[inv.Index] == nil { 230 b.invocationDeps[inv.Index] = make(map[uint64]bool) 231 } 232 b.invocationDeps[inv.Index][result.invIndex] = true 233 } 234 b.invocations[inv.Index] = inv 235 return true, nil 236 } 237 238 func (b *bigmachineExecutor) compile(ctx context.Context, m *sliceMachine, invIndex uint64) error { 239 b.mu.Lock() 240 // Traverse the invocation graph bottom-up, making sure everything on the 241 // machine is compiled. We produce a valid order, but we don't capture 242 // opportunities for parallel compilations. 243 // TODO(marius): allow for parallel compilation as some users are 244 // performing expensive computations inside of bigslice.Funcs. 245 var ( 246 todo = []uint64{invIndex} 247 invocations []execInvocation 248 ) 249 for len(todo) > 0 { 250 var i uint64 251 i, todo = todo[0], todo[1:] 252 invocations = append(invocations, b.invocations[i]) 253 for j := range b.invocationDeps[i] { 254 todo = append(todo, j) 255 } 256 } 257 b.mu.Unlock() 258 259 for i := len(invocations) - 1; i >= 0; i-- { 260 err := m.Compiles.Do(invocations[i].Index, func() error { 261 inv := invocations[i] 262 // Flatten these into lists so that we don't capture further 263 // structure by JSON encoding down the line. We also truncate them 264 // so that, e.g., huge lists of arguments don't make it into the trace. 265 args := make([]string, len(inv.Args)) 266 for i := range args { 267 args[i] = truncatef(inv.Args[i]) 268 } 269 b.sess.tracer.Event(m, inv, "B", "location", inv.Location, "args", args) 270 makeInvReader := func() (io.Reader, error) { 271 return b.invocationReader(inv.Index) 272 } 273 err := m.RetryCall(ctx, "Worker.Compile", makeInvReader, nil) 274 if err != nil { 275 b.sess.tracer.Event(m, inv, "E", "error", err) 276 } else { 277 b.sess.tracer.Event(m, inv, "E") 278 } 279 return err 280 }) 281 if err != nil { 282 return err 283 } 284 } 285 return nil 286 } 287 288 func (b *bigmachineExecutor) commit(ctx context.Context, m *sliceMachine, key string) error { 289 return m.Commits.Do(key, func() error { 290 return m.RetryCall(ctx, "Worker.CommitCombiner", TaskName{Op: key}, nil) 291 }) 292 } 293 294 func (b *bigmachineExecutor) Run(task *Task) { 295 task.Status.Print("waiting for a machine") 296 297 invIndex := task.Invocation.Index 298 added, err := b.addInvocation(task.Invocation) 299 if err != nil { 300 task.Errorf("error adding invocation %d to graph: %v", invIndex, err) 301 return 302 } 303 if added { 304 // This is the first time we are seeing this invocation. Check if we 305 // can serialize and cache the invocation that we will ultimately send 306 // to worker machines. Do this eagerly to discover errors quickly, 307 // rather than potentially waiting until machines are started. 308 if err := b.checkInvocationReader(invIndex); err != nil { 309 task.Errorf("error serializing invocation %d: %v", invIndex, err) 310 return 311 } 312 } 313 314 // Use the default/shared cluster unless the func is exclusive. 315 var cluster int 316 if task.Invocation.Exclusive { 317 cluster = int(invIndex) 318 } 319 mgr := b.manager(cluster) 320 procs := task.Pragma.Procs() 321 if task.Pragma.Exclusive() || procs > mgr.machprocs { 322 procs = mgr.machprocs 323 } 324 var ( 325 ctx = backgroundcontext.Get() 326 offerc, cancel = mgr.Offer(int(invIndex), procs) 327 m *sliceMachine 328 ) 329 select { 330 case <-ctx.Done(): 331 task.Error(ctx.Err()) 332 cancel() 333 return 334 case m = <-offerc: 335 } 336 numTasks := m.Stats.Int("tasks") 337 numTasks.Add(1) 338 m.UpdateStatus() 339 defer func() { 340 numTasks.Add(-1) 341 m.UpdateStatus() 342 }() 343 344 // Make sure that the invocation has been compiled on the selected 345 // machine. 346 compile: 347 for { 348 err := b.compile(ctx, m, invIndex) 349 switch { 350 case err == nil: 351 break compile 352 case ctx.Err() == nil && (err == context.Canceled || err == context.DeadlineExceeded): 353 // In this case, we've caught a context error from a prior 354 // invocation. We're going to try to run it again. Note that this 355 // is racy: the behavior remains correct but may imply additional 356 // data transfer. C'est la vie. 357 m.Compiles.Forget(invIndex) 358 case errors.Is(errors.Invalid, err) && errors.Match(fatalErr, err): 359 // Fatally invalid compilation parameters, e.g. func arguments that 360 // are not gob-encodable, are fatal to the task. 361 fallthrough 362 case errors.Is(errors.Remote, err): 363 // Compilations don't involve invoking user code, nor do they 364 // involve dependencies other than potentially uploading data from 365 // the driver node, so we consider any error to be fatal to the task. 366 task.Errorf("failed to compile invocation on machine %s: %v", m.Addr, err) 367 m.Done(procs, err) 368 return 369 default: 370 task.Status.Printf("task lost while compiling bigslice.Func: %v", err) 371 task.Set(TaskLost) 372 m.Done(procs, err) 373 return 374 } 375 } 376 377 // Populate the run request. Include the locations of all dependent 378 // outputs so that the receiving worker can read from them. 379 req := taskRunRequest{ 380 Name: task.Name, 381 Invocation: invIndex, 382 } 383 machineIndices := make(map[string]int) 384 g, _ := errgroup.WithContext(ctx) 385 for _, dep := range task.Deps { 386 for i := 0; i < dep.NumTask(); i++ { 387 deptask := dep.Task(i) 388 depm := b.location(deptask) 389 if depm == nil { 390 // TODO(marius): make this a separate state, or a separate 391 // error type? 392 task.Errorf("task %v has no location", deptask) 393 m.Done(procs, nil) 394 return 395 } 396 j, ok := machineIndices[depm.Addr] 397 if !ok { 398 j = len(machineIndices) 399 machineIndices[depm.Addr] = j 400 req.Machines = append(req.Machines, depm.Addr) 401 } 402 req.Locations = append(req.Locations, j) 403 key := dep.CombineKey 404 if key == "" { 405 continue 406 } 407 // Make sure that the result is committed. 408 g.Go(func() error { return b.commit(ctx, depm, key) }) 409 } 410 } 411 412 task.Status.Print(m.Addr) 413 if err := g.Wait(); err != nil { 414 task.Errorf("failed to commit combiner: %v", err) 415 return 416 } 417 418 // While we're running, also update task stats directly into the tasks's status. 419 // TODO(marius): also aggregate stats across all tasks. 420 statsCtx, statsCancel := context.WithCancel(ctx) 421 go monitorTaskStats(statsCtx, m, task) 422 423 b.sess.tracer.Event(m, task, "B") 424 task.Set(TaskRunning) 425 var reply taskRunReply 426 err = m.RetryCall(ctx, "Worker.Run", req, &reply) 427 statsCancel() 428 m.Done(procs, err) 429 switch { 430 case err == nil: 431 // Convert nanoseconds to microseconds to be same units as event durations. 432 b.sess.tracer.Event(m, task, "E", 433 "readDuration", reply.Vals["readDuration"]/1e3, 434 "writeDuration", reply.Vals["writeDuration"]/1e3, 435 ) 436 b.setLocation(task, m) 437 task.Status.Printf("done: %s", reply.Vals) 438 task.Scope.Reset(&reply.Scope) 439 task.Set(TaskOk) 440 m.Assign(task) 441 case ctx.Err() != nil: 442 b.sess.tracer.Event(m, task, "E", "error", ctx.Err()) 443 task.Error(err) 444 case errors.Is(errors.Remote, err) && errors.Match(fatalErr, err): 445 b.sess.tracer.Event(m, task, "E", "error", err, "error_type", "fatal") 446 // Fatal errors aren't retryable. 447 task.Error(err) 448 default: 449 // Everything else we consider as the task being lost. It'll get 450 // resubmitted by the evaluator. 451 b.sess.tracer.Event(m, task, "E", "error", err, "error_type", "lost") 452 task.Status.Printf("lost task during task evaluation: %v", err) 453 task.Set(TaskLost) 454 } 455 } 456 457 // monitorTaskStats monitors stats (e.g. records read/written) of the task 458 // running on m, updating task's status until ctx is done. 459 func monitorTaskStats(ctx context.Context, m *sliceMachine, task *Task) { 460 wait := func() { 461 select { 462 case <-time.After(statsPollInterval): 463 case <-ctx.Done(): 464 } 465 } 466 for ctx.Err() == nil { 467 var vals *stats.Values 468 err := m.RetryCall(ctx, "Worker.TaskStats", task.Name, &vals) 469 if err != nil { 470 log.Error.Printf("error getting task stats from %s: %v", m.Addr, err) 471 wait() 472 continue 473 } 474 task.Status.Printf("%s: %s", m.Addr, *vals) 475 wait() 476 } 477 } 478 479 func (b *bigmachineExecutor) Reader(task *Task, partition int) sliceio.ReadCloser { 480 m := b.location(task) 481 if m == nil { 482 return sliceio.NopCloser(sliceio.ErrReader(errors.E(errors.NotExist, fmt.Sprintf("task %s", task.Name)))) 483 } 484 if task.CombineKey != "" { 485 return sliceio.NopCloser(sliceio.ErrReader(fmt.Errorf("read %s: cannot read tasks with combine keys", task.Name))) 486 } 487 // TODO(marius): access the store here, too, in case it's a shared one (e.g., s3) 488 return newEvalReader(b, task, partition) 489 } 490 491 func (b *bigmachineExecutor) Discard(ctx context.Context, task *Task) { 492 if !task.Combiner.IsNil() && task.CombineKey != "" { 493 // We do not yet handle tasks with shared combiners. 494 return 495 } 496 task.Lock() 497 if task.state != TaskOk { 498 // We have no results to discard if the task is not TaskOk, as it has 499 // not completed successfully. 500 task.Unlock() 501 return 502 } 503 task.state = TaskRunning 504 task.Unlock() 505 m := b.location(task) 506 if m == nil { 507 return 508 } 509 m.Discard(ctx, task) 510 } 511 512 func (b *bigmachineExecutor) Eventer() eventlog.Eventer { 513 return b.sess.eventer 514 } 515 516 func (b *bigmachineExecutor) HandleDebug(handler *http.ServeMux) { 517 b.b.HandleDebug(handler) 518 } 519 520 // Location returns the machine on which the results of the provided 521 // task resides. 522 func (b *bigmachineExecutor) location(task *Task) *sliceMachine { 523 b.mu.Lock() 524 m := b.locations[task] 525 b.mu.Unlock() 526 return m 527 } 528 529 func (b *bigmachineExecutor) setLocation(task *Task, m *sliceMachine) { 530 b.mu.Lock() 531 b.locations[task] = m 532 b.mu.Unlock() 533 } 534 535 type combinerState int 536 537 const ( 538 combinerNone combinerState = iota 539 combinerWriting 540 combinerCommitted 541 combinerError 542 combinerIdle 543 // States > combinerIdle are reference counts. 544 ) 545 546 // A worker is the bigmachine service that runs individual tasks and serves 547 // the results of previous runs. Currently all output is buffered in memory. 548 type worker struct { 549 // MachineCombiners determines whether to use the MachineCombiners 550 // compilation option. 551 MachineCombiners bool 552 553 b *bigmachine.B 554 store Store 555 556 mu sync.Mutex 557 cond *ctxsync.Cond 558 compiles once.Map 559 tasks map[uint64]map[TaskName]*Task 560 taskStats map[uint64]map[TaskName]*stats.Map 561 slices map[uint64]bigslice.Slice 562 stats *stats.Map 563 564 // CombinerStates and combiners are used to manage shared combine 565 // buffers. combinerErrors is used to track the original cause of an 566 // a combiner error and report it accordingly. 567 combinerStates map[TaskName]combinerState 568 combinerErrors map[TaskName]error 569 combiners map[TaskName][]chan *combiner 570 571 commitLimiter *limiter.Limiter 572 } 573 574 func (w *worker) Init(b *bigmachine.B) error { 575 w.cond = ctxsync.NewCond(&w.mu) 576 w.tasks = make(map[uint64]map[TaskName]*Task) 577 w.taskStats = make(map[uint64]map[TaskName]*stats.Map) 578 w.slices = make(map[uint64]bigslice.Slice) 579 w.combiners = make(map[TaskName][]chan *combiner) 580 w.combinerStates = make(map[TaskName]combinerState) 581 w.combinerErrors = make(map[TaskName]error) 582 w.b = b 583 dir, err := ioutil.TempDir("", "bigslice") 584 if err != nil { 585 return err 586 } 587 w.store = &fileStore{Prefix: dir + "/"} 588 w.stats = stats.NewMap() 589 // Set up a limiter to limit the number of concurrent commits 590 // that are allowed to happen in the worker. 591 // 592 // TODO(marius): we should treat commits like tasks and apply 593 // load balancing/limiting instead. 594 w.commitLimiter = limiter.New() 595 procs := b.System().Maxprocs() 596 if procs == 0 { 597 procs = runtime.GOMAXPROCS(0) 598 } 599 w.commitLimiter.Release(procs) 600 return nil 601 } 602 603 // FuncLocations produces a slice of strings that describe the locations of 604 // Func creation. It is used to verify that this process is working from an 605 // identical Func registry. See bigslice.FuncLocations for more information. 606 func (w *worker) FuncLocations(ctx context.Context, _ struct{}, locs *[]string) error { 607 *locs = bigslice.FuncLocations() 608 return nil 609 } 610 611 // Compile compiles an invocation on the worker and stores the 612 // resulting tasks. Compile is idempotent: it will compile each 613 // invocation at most once. 614 func (w *worker) Compile(ctx context.Context, invReader io.Reader, _ *struct{}) (err error) { 615 defer func() { 616 if e := recover(); e != nil { 617 err = fmt.Errorf("invocation panic! %v", e) 618 err = errors.E(errors.Fatal, err) 619 } 620 }() 621 var inv execInvocation 622 if err = gob.NewDecoder(invReader).Decode(&inv); err != nil { 623 return errors.E(errors.Invalid, "error gob-decoding invocation", err) 624 } 625 return w.compiles.Do(inv.Index, func() error { 626 // Substitute invocation refs for the results of the invocation. 627 // The executor must ensure that all references have been compiled. 628 for i, arg := range inv.Args { 629 ref, ok := arg.(invocationRef) 630 if !ok { 631 continue 632 } 633 w.mu.Lock() 634 inv.Args[i], ok = w.slices[ref.Index] 635 w.mu.Unlock() 636 if !ok { 637 return fmt.Errorf("worker.Compile: invalid invocation reference %x", ref.Index) 638 } 639 } 640 slice := inv.Invoke() 641 tasks, err := compile(inv, slice, w.MachineCombiners) 642 if err != nil { 643 return err 644 } 645 all := make(map[*Task]bool) 646 for _, task := range tasks { 647 task.all(all) 648 } 649 named := make(map[TaskName]*Task) 650 for task := range all { 651 named[task.Name] = task 652 } 653 namedStats := make(map[TaskName]*stats.Map) 654 for task := range all { 655 namedStats[task.Name] = stats.NewMap() 656 } 657 w.mu.Lock() 658 w.tasks[inv.Index] = named 659 w.taskStats[inv.Index] = namedStats 660 w.slices[inv.Index] = &Result{Slice: slice, tasks: tasks} 661 w.mu.Unlock() 662 return nil 663 }) 664 } 665 666 // TaskRunRequest contains all data required to run an individual task. 667 type taskRunRequest struct { 668 // Invocation is the invocation from which the task was compiled. 669 Invocation uint64 670 671 // Name is the name of the task compiled from Invocation. 672 Name TaskName 673 674 // Machines stores the set of machines indexed in Locations. 675 Machines []string 676 677 // Locations indexes machine locations for task outputs. Locations 678 // stores the machine index of each task dependency. We rely on the 679 // fact that the task graph is identical to all viewers: locations 680 // are stored in the order of task dependencies. 681 Locations []int 682 } 683 684 func (r *taskRunRequest) location(taskIndex int) string { 685 return r.Machines[r.Locations[taskIndex]] 686 } 687 688 type taskRunReply struct { 689 // Vals are the stat values for the run of the task. 690 Vals stats.Values 691 692 // Scope is the scope of the task at completion time. 693 // TODO(marius): unify scopes with values, above. 694 Scope metrics.Scope 695 } 696 697 // maybeTaskFatalErr wraps errors in (*worker).Run that can cause fatal task 698 // errors, errors that will cause the evaluator to mark the task in TaskErr 699 // state and halt evaluation. This is generally used to identify (fatal) errors 700 // returned by application code. These errors are not returned from 701 // (*worker).Run; they are used internally to revise severity. 702 type maybeTaskFatalErr struct { 703 error 704 } 705 706 // reviseSeverity revises the severity of err for (*worker).Run. (*worker).Run 707 // only returns fatal errors for task fatal errors, errors that will cause tasks 708 // on the driver to be marked TaskErr and halt evaluation. 709 func reviseSeverity(err error) error { 710 if err == nil { 711 return nil 712 } 713 if e, ok := err.(maybeTaskFatalErr); ok { 714 return e.error 715 } 716 if e, ok := err.(*errors.Error); ok && e != nil && e.Severity == errors.Fatal { 717 // The error is fatal to this attempt to run the task but not fatal to 718 // the task overall, e.g. a fatal unavailable error when trying to read 719 // dependencies from other machines. We downgrade the error, so that the 720 // evaluator will retry. 721 e.Severity = errors.Unknown 722 return e 723 } 724 return err 725 } 726 727 // Run runs an individual task as described in the request. Run returns a nil 728 // error when the task was successfully run and its output deposited in a local 729 // buffer. If Run returns a *errors.Error with errors.Fatal severity, the task 730 // will be marked in TaskErr, and evaluation will halt. 731 func (w *worker) Run(ctx context.Context, req taskRunRequest, reply *taskRunReply) (err error) { 732 var task *Task 733 defer func() { 734 if e := recover(); e != nil { 735 stack := debug.Stack() 736 err = fmt.Errorf("panic while evaluating slice: %v\n%s", e, string(stack)) 737 err = maybeTaskFatalErr{errors.E(err, errors.Fatal)} 738 } 739 if err != nil { 740 log.Error.Printf("task %s error: %v", req.Name, err) 741 err = reviseSeverity(err) 742 if task != nil { 743 task.Error(errors.Recover(err)) 744 } 745 return 746 } 747 if task != nil { 748 task.Set(TaskOk) 749 } 750 }() 751 752 w.mu.Lock() 753 named := w.tasks[req.Invocation] 754 namedStats := w.taskStats[req.Invocation] 755 w.mu.Unlock() 756 if named == nil { 757 return maybeTaskFatalErr{errors.E(errors.Fatal, fmt.Errorf("invocation %x not compiled", req.Invocation))} 758 } 759 task = named[req.Name] 760 if task == nil { 761 return maybeTaskFatalErr{errors.E(errors.Fatal, fmt.Errorf("task %s not found", req.Name))} 762 } 763 taskStats := namedStats[req.Name] 764 ctx = metrics.ScopedContext(ctx, &task.Scope) 765 766 defer func() { 767 reply.Vals = make(stats.Values) 768 taskStats.AddAll(reply.Vals) 769 reply.Scope.Reset(&task.Scope) 770 }() 771 772 task.Lock() 773 switch task.state { 774 case TaskLost: 775 log.Printf("Worker.Run: %s: reviving LOST task", task.Name) 776 case TaskErr: 777 log.Printf("Worker.Run: %s: reviving FAILED task", task.Name) 778 case TaskInit: 779 default: 780 for task.state <= TaskRunning { 781 log.Printf("runtask: %s already running. Waiting for it to finish.", task.Name) 782 err = task.Wait(ctx) 783 if err != nil { 784 break 785 } 786 } 787 task.Unlock() 788 if e := task.Err(); e != nil { 789 err = e 790 } 791 return err 792 } 793 task.state = TaskRunning 794 task.Unlock() 795 // Gather inputs from the bigmachine cluster, dialing machines 796 // as necessary. 797 var ( 798 // Stats for the task. 799 taskTotalRecordsIn *stats.Int 800 taskRecordsIn *stats.Int 801 taskRecordsOut = taskStats.Int("write") 802 taskReadDuration = taskStats.Int("readDuration") 803 taskWriteDuration = taskStats.Int("writeDuration") 804 // Stats for the machine. 805 totalRecordsIn *stats.Int 806 recordsIn *stats.Int 807 recordsOut = w.stats.Int("write") 808 ) 809 taskRecordsOut.Set(0) 810 if len(task.Deps) > 0 { 811 taskTotalRecordsIn = taskStats.Int("inrecords") 812 taskTotalRecordsIn.Set(0) 813 taskRecordsIn = taskStats.Int("read") 814 taskRecordsIn.Set(0) 815 totalRecordsIn = w.stats.Int("inrecords") 816 recordsIn = w.stats.Int("read") 817 } 818 var ( 819 in = make([]sliceio.Reader, 0, len(task.Deps)) 820 taskIndex int 821 ) 822 for _, dep := range task.Deps { 823 // If the dependency has a combine key, they are combined on the 824 // machine, and we de-dup the dependencies. 825 // 826 // The caller of has already ensured that the combiner buffers 827 // are committed on the machines. 828 if dep.CombineKey != "" { 829 locations := make(map[string]bool) 830 for i := 0; i < dep.NumTask(); i++ { 831 addr := req.location(taskIndex) 832 taskIndex++ 833 // We only read the first combine key for each location. 834 // 835 // TODO(marius): compute some non-overlapping intersection of 836 // combine keys instead, so that we can handle error recovery 837 // properly. In particular, in the case of error recovery, we 838 // have to create new combiner keys so that they aren't written 839 // into previous combiner buffers. This suggests that combiner 840 // keys should be assigned by the executor, and not during 841 // compile time. 842 if locations[addr] { 843 continue 844 } 845 locations[addr] = true 846 } 847 for addr := range locations { 848 machine, err := w.b.Dial(ctx, addr) 849 if err != nil { 850 return err 851 } 852 r := newMachineReader(machine, taskPartition{TaskName{Op: dep.CombineKey}, dep.Partition}) 853 in = append(in, &statsReader{r, []*stats.Int{taskRecordsIn, recordsIn}, taskReadDuration}) 854 defer r.Close() 855 } 856 } else { 857 reader := new(multiReader) 858 reader.q = make([]sliceio.Reader, dep.NumTask()) 859 Tasks: 860 for j := 0; j < dep.NumTask(); j++ { 861 deptask := dep.Task(j) 862 // If we have it locally, or if we're using a shared backend store 863 // (e.g., S3), then read it directly. 864 info, err := w.store.Stat(ctx, deptask.Name, dep.Partition) 865 if err == nil { 866 rc, openErr := w.store.Open(ctx, deptask.Name, dep.Partition, 0) 867 if openErr == nil { 868 defer rc.Close() 869 r := sliceio.NewDecodingReader(rc) 870 reader.q[j] = &statsReader{r, []*stats.Int{taskRecordsIn, recordsIn}, taskReadDuration} 871 taskTotalRecordsIn.Add(info.Records) 872 totalRecordsIn.Add(info.Records) 873 taskIndex++ 874 continue Tasks 875 } 876 } 877 // Find the location of the task. 878 addr := req.location(taskIndex) 879 taskIndex++ 880 machine, err := w.b.Dial(ctx, addr) 881 if err != nil { 882 return err 883 } 884 tp := taskPartition{deptask.Name, dep.Partition} 885 if err := machine.RetryCall(ctx, "Worker.Stat", tp, &info); err != nil { 886 return err 887 } 888 r := newMachineReader(machine, tp) 889 reader.q[j] = &statsReader{r, []*stats.Int{taskRecordsIn, recordsIn}, taskReadDuration} 890 taskTotalRecordsIn.Add(info.Records) 891 totalRecordsIn.Add(info.Records) 892 defer r.Close() 893 } 894 // We shuffle the tasks here so that we don't encounter 895 // "thundering herd" issues were partitions are read sequentially 896 // from the same (ordered) list of machines. 897 // 898 // TODO(marius): possibly we should perform proper load balancing 899 // here 900 if DoShuffleReaders { 901 rand.Shuffle(len(reader.q), func(i, j int) { reader.q[i], reader.q[j] = reader.q[j], reader.q[i] }) 902 } 903 if dep.Expand { 904 in = append(in, reader.q...) 905 } else { 906 in = append(in, reader) 907 } 908 } 909 } 910 911 // If we have a combiner, then we partition globally for the machine 912 // into common combiners. 913 if !task.Combiner.IsNil() { 914 return w.runCombine(ctx, task, taskStats, task.Do(in)) 915 } 916 917 // Stream partition output directly to the underlying store, but 918 // through a buffer because the column encoder can make small 919 // writes. 920 // 921 // TODO(marius): switch to using a monotasks-like arrangement 922 // instead once we also have memory management, in order to control 923 // buffer growth. 924 type partition struct { 925 wc writeCommitter 926 buf *bufio.Writer 927 sliceio.Writer 928 } 929 partitions := make([]*partition, task.NumPartition) 930 for p := range partitions { 931 wc, err := w.store.Create(ctx, task.Name, p) 932 if err != nil { 933 return err 934 } 935 // TODO(marius): pool the writers so we can reuse them. 936 part := new(partition) 937 part.wc = wc 938 part.buf = bufio.NewWriter(wc) 939 part.Writer = &statsWriter{sliceio.NewEncodingWriter(part.buf), taskWriteDuration} 940 partitions[p] = part 941 } 942 defer func() { 943 for _, part := range partitions { 944 if part == nil { 945 continue 946 } 947 part.wc.Discard(ctx) 948 } 949 }() 950 out := task.Do(in) 951 count := make([]int64, task.NumPartition) 952 switch { 953 case task.NumOut() == 0: 954 // If there are no output columns, just drive the computation. 955 _, err := out.Read(ctx, frame.Empty) 956 if err != nil && err != sliceio.EOF { 957 return maybeTaskFatalErr{err} 958 } 959 return nil 960 case task.NumPartition > 1: 961 var psize = (*defaultChunksize + 99) / 100 962 var ( 963 partitionv = make([]frame.Frame, task.NumPartition) 964 lens = make([]int, task.NumPartition) 965 shards = make([]int, *defaultChunksize) 966 ) 967 for i := range partitionv { 968 partitionv[i] = frame.Make(task, psize, psize) 969 } 970 in := frame.Make(task, *defaultChunksize, *defaultChunksize) 971 for { 972 n, err := out.Read(ctx, in) 973 if err != nil && err != sliceio.EOF { 974 return maybeTaskFatalErr{err} 975 } 976 task.Partitioner(ctx, in, task.NumPartition, shards[:n]) 977 for i := 0; i < n; i++ { 978 p := shards[i] 979 j := lens[p] 980 frame.Copy(partitionv[p].Slice(j, j+1), in.Slice(i, i+1)) 981 lens[p]++ 982 count[p]++ 983 // Flush when we fill up. 984 if lens[p] == psize { 985 if writeErr := partitions[p].Write(ctx, partitionv[p]); writeErr != nil { 986 return maybeTaskFatalErr{errors.E(errors.Fatal, writeErr)} 987 } 988 lens[p] = 0 989 } 990 } 991 taskRecordsOut.Add(int64(n)) 992 recordsOut.Add(int64(n)) 993 if err == sliceio.EOF { 994 break 995 } 996 } 997 // Flush remaining data. 998 for p, n := range lens { 999 if n == 0 { 1000 continue 1001 } 1002 if err := partitions[p].Write(ctx, partitionv[p].Slice(0, n)); err != nil { 1003 return maybeTaskFatalErr{errors.E(errors.Fatal, err)} 1004 } 1005 } 1006 default: 1007 in := frame.Make(task, *defaultChunksize, *defaultChunksize) 1008 for { 1009 n, err := out.Read(ctx, in) 1010 if err != nil && err != sliceio.EOF { 1011 return maybeTaskFatalErr{err} 1012 } 1013 if writeErr := partitions[0].Write(ctx, in.Slice(0, n)); writeErr != nil { 1014 return maybeTaskFatalErr{errors.E(errors.Fatal, writeErr)} 1015 } 1016 taskRecordsOut.Add(int64(n)) 1017 recordsOut.Add(int64(n)) 1018 count[0] += int64(n) 1019 if err == sliceio.EOF { 1020 break 1021 } 1022 } 1023 } 1024 1025 for i, part := range partitions { 1026 if err := part.buf.Flush(); err != nil { 1027 return err 1028 } 1029 partitions[i] = nil 1030 if err := part.wc.Commit(ctx, count[i]); err != nil { 1031 return err 1032 } 1033 } 1034 partitions = nil 1035 return nil 1036 } 1037 1038 func (w *worker) Discard(ctx context.Context, taskName TaskName, _ *struct{}) (err error) { 1039 w.mu.Lock() 1040 named := w.tasks[taskName.InvIndex] 1041 w.mu.Unlock() 1042 if named == nil { 1043 return nil 1044 } 1045 task := named[taskName] 1046 if task == nil { 1047 return nil 1048 } 1049 task.Lock() 1050 if task.state != TaskOk { 1051 // We have no results to discard if the task is not TaskOk, as it has 1052 // not completed successfully. 1053 task.Unlock() 1054 return nil 1055 } 1056 task.state = TaskRunning 1057 task.Unlock() 1058 for partition := 0; partition < task.NumPartition; partition++ { 1059 err := w.store.Discard(ctx, taskName, partition) 1060 if err != nil { 1061 log.Printf("warning: discarding %v:%d: %v", taskName, partition, err) 1062 } 1063 } 1064 if !task.Combiner.IsNil() && task.CombineKey == "" { 1065 w.mu.Lock() 1066 w.combinerStates[task.Name] = combinerNone 1067 w.mu.Unlock() 1068 } 1069 task.Set(TaskLost) 1070 return nil 1071 } 1072 1073 // TaskStats returns the stats for the current or most recent run of a task on 1074 // w. This can be polled to display task status. 1075 func (w *worker) TaskStats(ctx context.Context, taskName TaskName, vals *stats.Values) error { 1076 w.mu.Lock() 1077 namedStats := w.taskStats[taskName.InvIndex] 1078 w.mu.Unlock() 1079 taskStats := namedStats[taskName] 1080 taskStats.AddAll(*vals) 1081 return nil 1082 } 1083 1084 func (w *worker) runCombine(ctx context.Context, task *Task, taskStats *stats.Map, 1085 in sliceio.Reader) (err error) { 1086 combineKey := task.Name 1087 if task.CombineKey != "" { 1088 combineKey = TaskName{Op: task.CombineKey} 1089 } 1090 w.mu.Lock() 1091 switch w.combinerStates[combineKey] { 1092 case combinerWriting: 1093 w.mu.Unlock() 1094 return fmt.Errorf("combine key %s still writing", combineKey) 1095 case combinerCommitted: 1096 w.mu.Unlock() 1097 return fmt.Errorf("combine key %s already committed", combineKey) 1098 case combinerError: 1099 combErr := w.combinerErrors[combineKey] 1100 w.mu.Unlock() 1101 return maybeTaskFatalErr{combErr} 1102 case combinerNone: 1103 combiners := make([]chan *combiner, task.NumPartition) 1104 for i := range combiners { 1105 comb, combErr := newCombiner(task, fmt.Sprintf("%s%d", combineKey, i), task.Combiner, *defaultChunksize*100) 1106 if combErr != nil { 1107 w.mu.Unlock() 1108 for j := 0; j < i; j++ { 1109 if discardErr := (<-combiners[j]).Discard(); discardErr != nil { 1110 log.Error.Printf("error discarding combiner: %v", discardErr) 1111 } 1112 } 1113 return combErr 1114 } 1115 combiners[i] = make(chan *combiner, 1) 1116 combiners[i] <- comb 1117 } 1118 w.combiners[combineKey] = combiners 1119 w.combinerStates[combineKey] = combinerIdle 1120 } 1121 w.combinerStates[combineKey]++ 1122 combiners := w.combiners[combineKey] 1123 w.mu.Unlock() 1124 1125 defer func() { 1126 w.mu.Lock() 1127 w.combinerStates[combineKey]-- 1128 w.mu.Unlock() 1129 if err == nil && task.CombineKey == "" { 1130 taskWriteDuration := taskStats.Int("writeDuration") 1131 start := time.Now() 1132 err = w.CommitCombiner(ctx, combineKey, nil) 1133 // Note that machine combiner write duration is not currently 1134 // captured, as it does not happen within the context of a single 1135 // task execution. 1136 taskWriteDuration.Add(time.Since(start).Nanoseconds()) 1137 } 1138 }() 1139 1140 var ( 1141 taskRecordsOut = taskStats.Int("write") 1142 recordsOut = w.stats.Int("write") 1143 ) 1144 // Now perform the partition-combine operation. We maintain a 1145 // per-task combine buffer for each partition. When this buffer 1146 // reaches half of its capacity, we attempt to combine up to 3/4ths 1147 // of its least frequent keys into the shared combine buffer. This 1148 // arrangement permits hot keys to be combined primarily in the task 1149 // buffer, while spilling less frequent keys into the per machine 1150 // buffer. (The local buffer is purely in memory, and has a fixed 1151 // capacity; the machine buffer spills to disk when it reaches a 1152 // preconfigured threshold.) 1153 var ( 1154 partitionCombiner = make([]*combiningFrame, task.NumPartition) 1155 out = frame.Make(task, *defaultChunksize, *defaultChunksize) 1156 shards = make([]int, *defaultChunksize) 1157 ) 1158 for i := range partitionCombiner { 1159 partitionCombiner[i] = makeCombiningFrame(task, task.Combiner, 8, 1) 1160 } 1161 for { 1162 n, err := in.Read(ctx, out) 1163 if err != nil && err != sliceio.EOF { 1164 return maybeTaskFatalErr{err} 1165 } 1166 task.Partitioner(ctx, out, task.NumPartition, shards[:n]) 1167 for i := 0; i < n; i++ { 1168 p := shards[i] 1169 pcomb := partitionCombiner[p] 1170 pcomb.Combine(out.Slice(i, i+1)) 1171 1172 len, cap := pcomb.Len(), pcomb.Cap() 1173 if len <= cap/2 { 1174 continue 1175 } 1176 var combiner *combiner 1177 if len >= 8 { 1178 combiner = <-combiners[p] 1179 } else { 1180 select { 1181 case combiner = <-combiners[p]: 1182 default: 1183 continue 1184 } 1185 } 1186 1187 flushed := pcomb.Compact() 1188 combErr := combiner.Combine(ctx, flushed) 1189 combiners[p] <- combiner 1190 if combErr != nil { 1191 return combErr 1192 } 1193 } 1194 taskRecordsOut.Add(int64(n)) 1195 recordsOut.Add(int64(n)) 1196 if err == sliceio.EOF { 1197 break 1198 } 1199 } 1200 // Flush the remainder. 1201 for p, comb := range partitionCombiner { 1202 combiner := <-combiners[p] 1203 err := combiner.Combine(ctx, comb.Compact()) 1204 combiners[p] <- combiner 1205 if err != nil { 1206 return err 1207 } 1208 } 1209 return nil 1210 } 1211 1212 func (w *worker) Stats(ctx context.Context, _ struct{}, values *stats.Values) error { 1213 w.stats.AddAll(*values) 1214 return nil 1215 } 1216 1217 // TaskPartition names a partition of a task. 1218 type taskPartition struct { 1219 // Name is the name of the task whose output is to be read. 1220 Name TaskName 1221 // Partition is the partition number to read. 1222 Partition int 1223 } 1224 1225 // Stat returns the SliceInfo for a slice. 1226 func (w *worker) Stat(ctx context.Context, tp taskPartition, info *sliceInfo) (err error) { 1227 *info, err = w.store.Stat(ctx, tp.Name, tp.Partition) 1228 return 1229 } 1230 1231 // CommitCombiner commits the current combiner buffer with the 1232 // provided key. After successful return, its results are available via 1233 // Read. 1234 func (w *worker) CommitCombiner(ctx context.Context, key TaskName, _ *struct{}) error { 1235 w.mu.Lock() 1236 defer w.mu.Unlock() 1237 for { 1238 switch w.combinerStates[key] { 1239 case combinerNone: 1240 return fmt.Errorf("invalid combiner key %s", key) 1241 case combinerWriting: 1242 if err := w.cond.Wait(ctx); err != nil { 1243 return err 1244 } 1245 case combinerCommitted: 1246 return nil 1247 case combinerError: 1248 return maybeTaskFatalErr{errors.E("error while writing combiner", w.combinerErrors[key])} 1249 case combinerIdle: 1250 w.combinerStates[key] = combinerWriting 1251 go w.writeCombiner(key) 1252 default: 1253 return fmt.Errorf("combiner key %s busy", key) 1254 } 1255 } 1256 } 1257 1258 func (w *worker) writeCombiner(key TaskName) { 1259 g, ctx := errgroup.WithContext(backgroundcontext.Get()) 1260 w.mu.Lock() 1261 defer w.mu.Unlock() 1262 for part := range w.combiners[key] { 1263 part := part 1264 combiner := <-w.combiners[key][part] 1265 g.Go(func() error { 1266 err := w.commitLimiter.Acquire(ctx, 1) 1267 if err != nil { 1268 return err 1269 } 1270 defer w.commitLimiter.Release(1) 1271 wc, err := w.store.Create(ctx, key, part) 1272 if err != nil { 1273 return err 1274 } 1275 buf := bufio.NewWriter(wc) 1276 enc := sliceio.NewEncodingWriter(buf) 1277 n, err := combiner.WriteTo(ctx, enc) 1278 if err != nil { 1279 wc.Discard(ctx) 1280 return err 1281 } 1282 if err := buf.Flush(); err != nil { 1283 wc.Discard(ctx) 1284 return err 1285 } 1286 return wc.Commit(ctx, n) 1287 }) 1288 } 1289 w.mu.Unlock() 1290 err := g.Wait() 1291 w.mu.Lock() 1292 w.combiners[key] = nil 1293 if err == nil { 1294 w.combinerStates[key] = combinerCommitted 1295 } else { 1296 log.Error.Printf("failed to write combine buffer %s: %v", key, err) 1297 w.combinerErrors[key] = err 1298 w.combinerStates[key] = combinerError 1299 } 1300 w.cond.Broadcast() 1301 } 1302 1303 // Read reads a slice. 1304 // 1305 // TODO(marius): should we flush combined outputs explicitly? 1306 func (w *worker) Read(ctx context.Context, req readRequest, rc *io.ReadCloser) (err error) { 1307 *rc, err = w.store.Open(ctx, req.Name, req.Partition, req.Offset) 1308 return 1309 } 1310 1311 // readRequest is the request payload for Worker.Run 1312 type readRequest struct { 1313 // Name is the name of the task whose output is to be read. 1314 Name TaskName 1315 // Partition is the partition number to read. 1316 Partition int 1317 // Offset is the start offset of the read 1318 Offset int64 1319 } 1320 1321 // openerAt opens an io.ReadCloser at a given offset. This is used to 1322 // reestablish io.ReadClosers when they are lost due to potentially recoverable 1323 // errors. 1324 type openerAt interface { 1325 // Open returns a new io.ReadCloser. 1326 OpenAt(ctx context.Context, offset int64) (io.ReadCloser, error) 1327 } 1328 1329 // retryReader implements an io.ReadCloser that is backed by an openerAt. If it 1330 // encounters an error, it retries by using the openerAt to reopen a new 1331 // io.ReadCloser. 1332 type retryReader struct { 1333 ctx context.Context 1334 // openerAt is used to open and reopen the backing io.ReadCloser. 1335 openerAt openerAt 1336 1337 err error 1338 reader io.ReadCloser 1339 bytes int64 1340 retries int 1341 } 1342 1343 func newRetryReader(ctx context.Context, openerAt openerAt) *retryReader { 1344 return &retryReader{ 1345 ctx: ctx, 1346 openerAt: openerAt, 1347 } 1348 } 1349 1350 // Read implements io.Reader. 1351 func (r *retryReader) Read(data []byte) (int, error) { 1352 for { 1353 if r.err != nil { 1354 return 0, r.err 1355 } 1356 n, err := func() (int, error) { 1357 if r.reader == nil { 1358 if r.retries > 0 { 1359 log.Printf("reader %v: retrying(%d) from offset %d", 1360 r.openerAt, r.retries, r.bytes) 1361 } 1362 var err error 1363 r.reader, err = r.openerAt.OpenAt(r.ctx, r.bytes) 1364 if err != nil { 1365 return 0, err 1366 } 1367 } 1368 return r.reader.Read(data) 1369 }() 1370 if err == nil || err == io.EOF { 1371 // We successfully read (and opened, if that was necessary). 1372 r.retries = 0 1373 r.err = err 1374 r.bytes += int64(n) 1375 return n, err 1376 } 1377 // Here, we blindly retry regardless of error kind/severity. 1378 // This allows us to retry on errors such as aws-sdk or io.UnexpectedEOF. 1379 // The subsequent call to Worker.Read will detect any permanent 1380 // errors in any case. 1381 log.Error.Printf("reader %v: error(retry %d) at %d bytes: %v", 1382 r.openerAt, r.retries, r.bytes, err) 1383 if r.reader != nil { 1384 // Nothing useful we can do on failed Close. 1385 _ = r.reader.Close() 1386 r.reader = nil 1387 } 1388 r.retries++ 1389 if r.err = retry.Wait(r.ctx, retryPolicy, r.retries); r.err != nil { 1390 return 0, r.err 1391 } 1392 } 1393 } 1394 1395 func (r *retryReader) Close() error { 1396 if r.reader == nil { 1397 return nil 1398 } 1399 err := r.reader.Close() 1400 r.reader = nil 1401 return err 1402 } 1403 1404 // openerAtReader is a sliceio.Reader that is backed by an OpenerAt used to 1405 // open a reader and retry on error. For example, we may reopen a connection 1406 // to a machine that experienced a temporary network error, synced to where we 1407 // have successfully read so far. 1408 type openerAtReader struct { 1409 // OpenerAt is used to open readers (and reopen on error). 1410 OpenerAt openerAt 1411 // ReviseSeverity indicates whether the severity of errors returned from 1412 // Read will be revised with respect to task errors (i.e. should errors be 1413 // considered task-fatal?). 1414 ReviseSeverity bool 1415 1416 readCloser io.ReadCloser 1417 sliceioReader sliceio.Reader 1418 } 1419 1420 // Read implements sliceio.Reader. 1421 func (r *openerAtReader) Read(ctx context.Context, f frame.Frame) (int, error) { 1422 if r.readCloser == nil { 1423 r.readCloser = newRetryReader(ctx, r.OpenerAt) 1424 r.sliceioReader = sliceio.NewDecodingReader(r.readCloser) 1425 } 1426 n, err := r.sliceioReader.Read(ctx, f) 1427 if r.ReviseSeverity { 1428 err = reviseSeverity(err) 1429 } 1430 return n, err 1431 } 1432 1433 // Close implements io.Closer. 1434 func (r *openerAtReader) Close() error { 1435 if r.readCloser == nil { 1436 return nil 1437 } 1438 r.sliceioReader = nil 1439 err := r.readCloser.Close() 1440 r.readCloser = nil 1441 return err 1442 } 1443 1444 // machineTaskPartition is a task partition on a specific machine. It implements 1445 // the openerAt interface to provide an io.ReadCloser to read the task data. 1446 type machineTaskPartition struct { 1447 // Machine is the machine from which task data is read. 1448 Machine *bigmachine.Machine 1449 // TaskPartition is the task and partition that should be read. 1450 TaskPartition taskPartition 1451 } 1452 1453 // OpenAt implements openerAt. 1454 func (m machineTaskPartition) OpenAt(ctx context.Context, offset int64) (io.ReadCloser, error) { 1455 var r io.ReadCloser 1456 err := m.Machine.RetryCall(ctx, "Worker.Read", 1457 readRequest{m.TaskPartition.Name, m.TaskPartition.Partition, offset}, &r) 1458 return r, err 1459 } 1460 1461 func (m machineTaskPartition) String() string { 1462 return fmt.Sprintf("Worker.Read %s:%s:%d", m.Machine.Addr, m.TaskPartition.Name, m.TaskPartition.Partition) 1463 } 1464 1465 // newMachineReader returns a reader that reads a taskPartition from a machine. 1466 // It issues the (streaming) read RPC on the first call to Read so that data 1467 // are not buffered unnecessarily. 1468 func newMachineReader(machine *bigmachine.Machine, taskPartition taskPartition) *openerAtReader { 1469 return &openerAtReader{ 1470 OpenerAt: machineTaskPartition{ 1471 Machine: machine, 1472 TaskPartition: taskPartition, 1473 }, 1474 // This is how all slice operations read data to process and does not 1475 // involve application code. By revising the severity, we save slice 1476 // operation implementations from each individually revising the 1477 // severity of machine reads. 1478 ReviseSeverity: true, 1479 } 1480 } 1481 1482 // evalOpenerAt is an openerAt that opens a reader for a task partition, first 1483 // evaluating the task to ensure that it is available. It is used for fault 1484 // tolerance of post-evaluation reads. 1485 type evalOpenerAt struct { 1486 // Executor is the executor used to execute the task before opening the 1487 // reader. 1488 Executor *bigmachineExecutor 1489 // Task is the task whose data is read by the returned reader. 1490 Task *Task 1491 // Partition is the data partition read by the returned reader. 1492 Partition int 1493 1494 // machine is the machine used by the last attempt to open a reader, 1495 // post-successful evaluation. 1496 machine *bigmachine.Machine 1497 } 1498 1499 // OpenAt implements openerAt. 1500 func (e *evalOpenerAt) OpenAt(ctx context.Context, offset int64) (io.ReadCloser, error) { 1501 // Evaluate the task, so that results are available for reading. This 1502 // provides some fault tolerance when machines are lost after evaluation 1503 // is complete (e.g. during final result scanning). 1504 err := Eval(ctx, e.Executor, []*Task{e.Task}, nil) 1505 if err != nil { 1506 return nil, err 1507 } 1508 e.machine = e.Executor.location(e.Task).Machine 1509 var r io.ReadCloser 1510 err = e.machine.RetryCall(ctx, 1511 "Worker.Read", readRequest{e.Task.Name, e.Partition, offset}, &r) 1512 return r, err 1513 } 1514 1515 func (e evalOpenerAt) String() string { 1516 addr := "<no machine yet>" 1517 if e.machine != nil { 1518 addr = e.machine.Addr 1519 } 1520 return fmt.Sprintf("Worker.Read %s:%s:%d", addr, e.Task.Name, e.Partition) 1521 } 1522 1523 // newEvalReader returns a reader that reads the data of the given task and 1524 // partition. It attempts to evaluate the task before reading. It is used for 1525 // fault tolerance of post-evaluation reads. 1526 func newEvalReader(executor *bigmachineExecutor, task *Task, partition int) *openerAtReader { 1527 return &openerAtReader{ 1528 OpenerAt: &evalOpenerAt{ 1529 Executor: executor, 1530 Task: task, 1531 Partition: partition, 1532 }, 1533 ReviseSeverity: false, 1534 } 1535 } 1536 1537 type statsReader struct { 1538 reader sliceio.Reader 1539 // numRead is a slice of *stats.Int, each of which is incremented with the 1540 // number of records read. 1541 numRead []*stats.Int 1542 // readDurationNs is the total amount of time taken by the Read call to the 1543 // underlying reader in nanoseconds. 1544 readDurationNs *stats.Int 1545 } 1546 1547 func (s *statsReader) Read(ctx context.Context, f frame.Frame) (int, error) { 1548 start := time.Now() 1549 n, err := s.reader.Read(ctx, f) 1550 s.readDurationNs.Add(time.Since(start).Nanoseconds()) 1551 for _, istat := range s.numRead { 1552 istat.Add(int64(n)) 1553 } 1554 return n, err 1555 } 1556 1557 func truncatef(v interface{}) string { 1558 b := limitbuf.NewLogger(512, 1559 // fmt.Fprint buffers the entire string in memory before writing it to b. 1560 // truncatef returns a small result, as intended, but potentially uses a lot 1561 // of memory transiently. This logging may inform users who are experiencing 1562 // this, because they may otherwise see a lot of driver memory and CPU usage 1563 // for no apparent purpose. 1564 // TODO: Use O(1) space. 1565 limitbuf.LogIfTruncatingMaxMultiple(100)) 1566 fmt.Fprint(b, v) 1567 return b.String() 1568 } 1569 1570 type statsWriter struct { 1571 writer sliceio.Writer 1572 writeDurationNs *stats.Int 1573 } 1574 1575 func (s *statsWriter) Write(ctx context.Context, f frame.Frame) error { 1576 start := time.Now() 1577 err := s.writer.Write(ctx, f) 1578 s.writeDurationNs.Add(time.Since(start).Nanoseconds()) 1579 return err 1580 }