github.com/grailbio/bigslice@v0.0.0-20230519005545-30c4c12152ad/exec/eval.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 implements compilation, evaluation, and execution of 6 // Bigslice slice operations. 7 package exec 8 9 import ( 10 "context" 11 "fmt" 12 "net/http" 13 "sync" 14 "time" 15 16 "github.com/grailbio/base/errors" 17 "github.com/grailbio/base/eventlog" 18 "github.com/grailbio/base/log" 19 "github.com/grailbio/base/status" 20 "github.com/grailbio/bigslice/internal/defaultsize" 21 "github.com/grailbio/bigslice/sliceio" 22 ) 23 24 var defaultChunksize = &defaultsize.Chunk 25 26 // maxConsecutiveLost is the maximum number of times a task can be run and lost 27 // consecutively before we give up and consider it an error. This helps catch 28 // persistent errors that prevent meaningful progress from being made in an 29 // evaluation (e.g. an error that causes worker processes to exit). 30 const maxConsecutiveLost = 5 31 32 // enableMaxConsecutiveLost enables the use of the maxConsecutiveLost value to 33 // consider repeatedly lost tasks as errors. See documentation for 34 // maxConsecutiveLost. It is exposed so that we can disable it in some testing 35 // scenarios. 36 var enableMaxConsecutiveLost = true 37 38 // Executor defines an interface used to provide implementations of 39 // task runners. An Executor is responsible for running single tasks, 40 // partitioning their outputs, and instantiating readers to retrieve the 41 // output of any given task. 42 type Executor interface { 43 // Name returns a human-friendly name for this executor. 44 Name() string 45 46 // Start starts the executor. It is called before evaluation has started 47 // and after all funcs have been registered. Start need not return: 48 // for example, the Bigmachine implementation of Executor uses 49 // Start as an entry point for worker processes. 50 Start(*Session) (shutdown func()) 51 52 // Run runs a task. The executor sets the state of the task as it 53 // progresses. The task should enter in state TaskWaiting; by the 54 // time Run returns the task state is >= TaskOk. 55 Run(*Task) 56 57 // Reader returns a locally accessible ReadCloser for the requested task. 58 Reader(*Task, int) sliceio.ReadCloser 59 60 // Discard discards the storage resources held by a computed task. 61 // Discarding is best-effort, so no error is returned. 62 Discard(context.Context, *Task) 63 64 // Eventer returns the eventer used to log events relevant to this executor. 65 Eventer() eventlog.Eventer 66 67 // HandleDebug adds executor-specific debug handlers to the provided 68 // http.ServeMux. This is used to serve diagnostic information relating 69 // to the executor. 70 HandleDebug(handler *http.ServeMux) 71 } 72 73 // Eval simultaneously evaluates a set of task graphs from the provided set of 74 // roots. Eval uses the provided executor to dispatch tasks when their 75 // dependencies have been satisfied. Eval returns on evaluation error or else 76 // when all roots are fully evaluated. 77 // 78 // TODO(marius): we can often stream across shuffle boundaries. This would 79 // complicate scheduling, but may be worth doing. 80 func Eval(ctx context.Context, executor Executor, roots []*Task, group *status.Group) error { 81 ctx, cancel := context.WithCancel(ctx) 82 defer cancel() 83 var ( 84 state = newState() 85 evalStatus = newEvalStatus(group) 86 donec = make(chan *Task, 8) 87 errc = make(chan error) 88 ) 89 for { 90 for _, task := range roots { 91 state.Enqueue(task) 92 } 93 if state.Done() { 94 return state.Err() 95 } 96 for !state.Done() && !state.Todo() { 97 select { 98 case err := <-errc: 99 if err == nil { 100 panic("nil err") 101 } 102 return err 103 case task := <-donec: 104 state.Return(task) 105 evalStatus.markDone(task) 106 } 107 } 108 // Mark each ready task as runnable and keep track of them. 109 // The executor manages parallelism. 110 for _, task := range state.Runnable() { 111 task.Lock() 112 if task.state == TaskLost { 113 log.Printf("evaluator: resubmitting lost task %v", task) 114 task.state = TaskInit 115 } 116 status := group.Start(task.Name) 117 evalStatus.markWaiting(task) 118 // runner is true if this evaluator is going to execute the task. 119 runner := task.state == TaskInit 120 var startRunTime time.Time 121 if runner { 122 task.state = TaskWaiting 123 task.Status = status 124 startRunTime = time.Now() 125 go executor.Run(task) 126 } else { 127 status.Print("running in another invocation") 128 } 129 go func(task *Task) { 130 var err error 131 for task.state < TaskRunning && err == nil { 132 err = task.Wait(ctx) 133 } 134 evalStatus.markRunning(task) 135 for task.state < TaskOk && err == nil { 136 err = task.Wait(ctx) 137 } 138 if runner { 139 if enableMaxConsecutiveLost { 140 // Only the runner bookkeeps consecutiveLost to avoid 141 // double-counting task loss. 142 switch task.state { 143 case TaskOk: 144 task.consecutiveLost = 0 145 case TaskLost: 146 task.consecutiveLost++ 147 if task.consecutiveLost >= maxConsecutiveLost { 148 // We've lost this task too many times, so we 149 // consider it in error. 150 task.state = TaskErr 151 task.err = errors.E( 152 errors.TooManyTries, 153 fmt.Sprintf("lost on %d consecutive attempts", task.consecutiveLost), 154 ) 155 task.Status.Printf(task.err.Error()) 156 task.Broadcast() 157 } 158 } 159 } 160 d := time.Since(startRunTime) 161 executor.Eventer().Event("bigslice:taskComplete", 162 "name", task.Name.String(), 163 "state", task.state.String(), 164 "duration", d.Nanoseconds()/1e6) 165 } 166 task.Unlock() 167 status.Done() 168 if err != nil { 169 errc <- err 170 } else { 171 donec <- task 172 } 173 }(task) 174 } 175 } 176 } 177 178 type ( 179 // evalStatus handles group status printing for an evaluation. We print 180 // stats on waiting and running tasks, the state for which is kept here. 181 // This is redundant with state held in the task graph itself, but it's a 182 // little simpler to piggyback on the evaluation logic: we don't 183 // have to take locks on the tasks, we already wait for state changes, etc. 184 evalStatus struct { 185 // group is the status group of the evaluation that we manipulate. 186 group *status.Group 187 188 // mu protects the task map. 189 mu sync.Mutex 190 // tasks holds the state relevant to status printing of relevant (read: 191 // scheduled) tasks. 192 tasks map[*Task]evalStatusState 193 } 194 195 // evalStatusState is the state of a task in the context of group status 196 // display. 197 evalStatusState int 198 ) 199 200 const ( 201 evalStatusWaiting evalStatusState = iota 202 evalStatusRunning 203 ) 204 205 func newEvalStatus(group *status.Group) *evalStatus { 206 s := evalStatus{ 207 group: group, 208 tasks: make(map[*Task]evalStatusState), 209 } 210 s.lockedPrint() 211 return &s 212 } 213 214 func (s *evalStatus) markWaiting(t *Task) { 215 s.mu.Lock() 216 defer s.mu.Unlock() 217 s.tasks[t] = evalStatusWaiting 218 s.lockedPrint() 219 } 220 221 func (s *evalStatus) markRunning(t *Task) { 222 s.mu.Lock() 223 defer s.mu.Unlock() 224 s.tasks[t] = evalStatusRunning 225 s.lockedPrint() 226 } 227 228 func (s *evalStatus) markDone(t *Task) { 229 s.mu.Lock() 230 defer s.mu.Unlock() 231 delete(s.tasks, t) 232 s.lockedPrint() 233 } 234 235 // lockedPrint prints the current status to the status group. s.tasks must not 236 // be concurrently modified. 237 func (s *evalStatus) lockedPrint() { 238 var ( 239 waiting int 240 running int 241 ) 242 for _, s := range s.tasks { 243 switch s { 244 case evalStatusWaiting: 245 waiting++ 246 case evalStatusRunning: 247 running++ 248 default: 249 panic("unknown status") 250 } 251 } 252 s.group.Printf("waiting/running: %d/%d", waiting, running) 253 } 254 255 // State maintains state for the task graph being run by the 256 // evaluator. It maintains per-node waitlists so that it can 257 // efficiently traverse only the required portion of the task graph 258 // for each task update. When a task's waitlist has been cleared, 259 // state re-traverses the graph from that task. This reconciles task 260 // changes that have occurred between updates. This may cause a task 261 // to be re-queued, e.g., if a dependent task changed status from TaskOk 262 // to TaskLost. State does not watch for task changes for tasks that 263 // are ready, thus it won't aggressively recompute a lost task that 264 // is going to be needed by another task with a nonzero waitlist. 265 // This is only discovered once that waitlist is drained. The scheme 266 // could be more aggressive in this case, but these cases should be 267 // rare enough to not warrant the added complexity. 268 // 269 // In order to ensure that the state operates on consistent view of 270 // the task graph, waitlist decisions are memoized per toplevel call; 271 // it does not require locking subgraphs. 272 type state struct { 273 // deps and counts maintains the task waitlist. 274 deps map[*Task]map[*Task]struct{} 275 counts map[*Task]int 276 277 // todo is the set of tasks that are scheduled to be run. They are 278 // retrieved via the Runnable method. 279 todo map[*Task]bool 280 281 // pending is the set of tasks that have been scheduled but have not 282 // yet been returned via Done. 283 pending map[*Task]bool 284 285 // wait stores memoized task waiting count (based on a single 286 // atomic reading of task state), per round. This is what enables 287 // state to maintain a consistent view of the task graph state. 288 wait map[*Task]int 289 290 err error 291 } 292 293 // newState returns a newly allocated, empty state. 294 func newState() *state { 295 return &state{ 296 deps: make(map[*Task]map[*Task]struct{}), 297 counts: make(map[*Task]int), 298 todo: make(map[*Task]bool), 299 pending: make(map[*Task]bool), 300 wait: make(map[*Task]int), 301 } 302 } 303 304 // Enqueue enqueues all ready tasks in the provided task graph, 305 // traversing only as much of it as necessary to schedule all 306 // currently runnable tasks in the graph. Enqueue maintains the 307 // waiting state for tasks so the correct (and minimal) task graphs 308 // can be efficiently enqueued on task completion. 309 // 310 // Enqueue understands the phase structure of the task graph, 311 // allowing it to skip fine-grained dependency maintenance across 312 // shuffle dependencies. Instead, for such dependencies, it keeps 313 // track of dependencies across task phases (i.e., groups of tasks 314 // that must all be done until we can schedule the next group), and 315 // maintaining simple counts of the number of dependencies satisfied. 316 // This allows scheduling to be done in O(Ntasks) instead of 317 // O(Nedges). Nedges in turn is quadratic in the number of tasks when 318 // there are shuffle dependencies. 319 func (s *state) Enqueue(task *Task) (nwait int) { 320 if n, ok := s.wait[task.Head()]; ok { 321 return n 322 } 323 for _, task := range task.Phase() { 324 switch task.State() { 325 case TaskOk, TaskErr: 326 case TaskWaiting, TaskRunning: 327 s.schedule(task) 328 nwait++ 329 case TaskInit, TaskLost: 330 s.clear(task) 331 ready := true 332 for _, dep := range task.Deps { 333 n := s.Enqueue(dep.Head) 334 if n == 0 { 335 continue 336 } 337 s.add(dep.Head, task, n) 338 ready = false 339 } 340 nwait++ 341 if ready { 342 s.schedule(task) 343 } 344 } 345 } 346 s.wait[task.Head()] = nwait 347 return 348 } 349 350 // Return returns a pending task to state, recomputing the state view 351 // and scheduling follow-on tasks. 352 func (s *state) Return(task *Task) { 353 if !s.pending[task] { 354 panic("exec.Eval: done task " + task.Name.String() + ": not pending") 355 } 356 // Clear the wait map between each call since the state of tasks may 357 // have changed between calls. 358 s.wait = make(map[*Task]int) 359 delete(s.pending, task) 360 switch task.State() { 361 default: 362 // We might be racing with another evaluator. Reschedule until 363 // we get into an actionable state. 364 s.schedule(task) 365 case TaskErr: 366 msg := fmt.Sprintf("error running %s", task.Name) 367 s.err = errors.E(msg, task.err) 368 case TaskOk: 369 for _, task := range s.done(task.Head()) { 370 s.Enqueue(task) 371 } 372 case TaskLost: 373 // Re-enqueue immediately. 374 s.Enqueue(task) 375 } 376 } 377 378 // Runnable returns the current set of runnable tasks and 379 // resets the todo list. It is called by Eval to schedule a batch 380 // of tasks. 381 func (s *state) Runnable() (tasks []*Task) { 382 if len(s.todo) == 0 { 383 return 384 } 385 tasks = make([]*Task, 0, len(s.todo)) 386 for task := range s.todo { 387 tasks = append(tasks, task) 388 delete(s.todo, task) 389 s.pending[task] = true 390 } 391 return 392 } 393 394 // Todo returns whether state has tasks to be scheduled. 395 func (s *state) Todo() bool { 396 return len(s.todo) > 0 397 } 398 399 // Done returns whether evaluation is done. Evaluation is done when 400 // there remain no pending tasks, or tasks to be scheduled. Evaluation 401 // is also done if an error has occurred. 402 func (s *state) Done() bool { 403 return s.err != nil || len(s.todo) == 0 && len(s.pending) == 0 404 } 405 406 // Err returns an error, if any, that occurred during evaluation. 407 func (s *state) Err() error { 408 return s.err 409 } 410 411 // Schedule schedules the provided task. It is a no-op if 412 // the task has already been scheduled or is pending. 413 func (s *state) schedule(task *Task) { 414 if s.pending[task] { 415 return 416 } 417 s.todo[task] = true 418 } 419 420 // Clear the dependency information stored for task. 421 func (s *state) clear(task *Task) { 422 delete(s.counts, task) 423 for _, dep := range task.Deps { 424 if d := s.deps[dep.Head]; d != nil { 425 delete(d, task) 426 } 427 } 428 } 429 430 // Add adds a dependency from the provided src to dst tasks. 431 func (s *state) add(src, dst *Task, n int) { 432 if d := s.deps[src]; d == nil { 433 s.deps[src] = map[*Task]struct{}{dst: {}} 434 s.counts[dst] += n 435 } else if _, ok := d[dst]; !ok { 436 d[dst] = struct{}{} 437 s.counts[dst] += n 438 } 439 } 440 441 // Done marks the provided task as done, and returns the set 442 // of tasks that have consequently become ready for evaluation. 443 func (s *state) done(src *Task) (ready []*Task) { 444 for dst := range s.deps[src] { 445 s.counts[dst]-- 446 if s.counts[dst] == 0 { 447 ready = append(ready, dst) 448 } 449 } 450 return 451 }