github.com/grailbio/bigslice@v0.0.0-20230519005545-30c4c12152ad/exec/slicemachine.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 "container/heap" 9 "context" 10 "fmt" 11 "sync" 12 "time" 13 14 "github.com/grailbio/base/backgroundcontext" 15 "github.com/grailbio/base/data" 16 "github.com/grailbio/base/errors" 17 "github.com/grailbio/base/log" 18 "github.com/grailbio/base/status" 19 "github.com/grailbio/base/sync/once" 20 "github.com/grailbio/bigmachine" 21 "github.com/grailbio/bigslice" 22 "github.com/grailbio/bigslice/stats" 23 "golang.org/x/sync/errgroup" 24 ) 25 26 // ProbationTimeout is the amount of time that a machine will 27 // remain in probation without being explicitly marked healthy. 28 var ProbationTimeout = 30 * time.Second 29 30 // maxStartMachines is the maximum number of machines that 31 // may be started in one batch. 32 const maxStartMachines = 10 33 34 // MachineHealth is the overall assessment of machine health by 35 // the bigmachine executor. 36 type machineHealth int 37 38 const ( 39 machineOk machineHealth = iota 40 machineProbation 41 machineLost 42 ) 43 44 // SliceMachine manages a single bigmachine.Machine instance. 45 type sliceMachine struct { 46 *bigmachine.Machine 47 48 // Compiles ensures that each invocation is compiled exactly once on 49 // the machine. 50 Compiles once.Map 51 52 // Commits keeps track of which combine keys have been committed 53 // on the machine, so that they are run exactly once on the machine. 54 Commits once.Map 55 56 Stats *stats.Map 57 Status *status.Task 58 59 // maxTaskProcs is the maximum number of procs on the machine to which tasks 60 // can be assigned. This can be different from Maxprocs, as it is attenuated 61 // by (*machineManager).Maxload. 62 maxTaskProcs int 63 64 // taskProcs is the current number of procs on the machine that have tasks 65 // assigned. taskProcs is managed by the machineManager. 66 taskProcs int 67 68 // health is managed by the machineManager. 69 health machineHealth 70 71 // lastFailure is managed by the machineManager. 72 lastFailure time.Time 73 74 // index is the machine's index in the executor's priority queue. 75 index int 76 77 // donec is the channel to which we send messages reporting that this 78 // machine is done using some procs. 79 donec chan machineDone 80 81 mu sync.Mutex 82 83 // Lost indicates whether the machine is considered lost as per 84 // bigmachine. 85 lost bool 86 87 // Tasks is the set of tasks that have been run on this machine. 88 // It is used to mark tasks lost when a machine fails. 89 tasks map[*Task]struct{} 90 91 disk bigmachine.DiskInfo 92 mem bigmachine.MemInfo 93 load bigmachine.LoadInfo 94 vals stats.Values 95 } 96 97 func (s *sliceMachine) String() string { 98 var health string 99 switch s.health { 100 case machineOk: 101 health = "ok" 102 case machineProbation: 103 health = "probation" 104 case machineLost: 105 health = "lost" 106 } 107 return fmt.Sprintf("%s (%s)", s.Addr, health) 108 } 109 110 // Done returns procs on the machine, and reports any error observed while 111 // running tasks. 112 func (s *sliceMachine) Done(procs int, err error) { 113 s.donec <- machineDone{s, procs, err} 114 } 115 116 // Assign assigns the provided task to this machine. If the machine 117 // fails, its assigned tasks are marked LOST. 118 func (s *sliceMachine) Assign(task *Task) { 119 s.mu.Lock() 120 defer s.mu.Unlock() 121 if s.lost { 122 task.Set(TaskLost) 123 } else { 124 s.tasks[task] = struct{}{} 125 } 126 } 127 128 // Discard discards the storage resources held by task. The task will be 129 // unassigned from s and considered TaskLost. If s does not own task, no-op. 130 func (s *sliceMachine) Discard(ctx context.Context, task *Task) { 131 s.mu.Lock() 132 _, ok := s.tasks[task] 133 delete(s.tasks, task) 134 s.mu.Unlock() 135 if !ok { 136 return 137 } 138 // s exclusively owns task's state during this time, so this does not race 139 // with anything else. 140 task.Set(TaskLost) 141 if err := s.RetryCall(ctx, "Worker.Discard", task.Name, nil); err != nil { 142 log.Error.Printf("error discarding %v: %v", task, err) 143 } 144 } 145 146 // Go manages a sliceMachine: it polls stats at regular intervals and 147 // marks tasks as lost when a machine fails. 148 func (s *sliceMachine) Go(ctx context.Context) { 149 stopped := s.Wait(bigmachine.Stopped) 150 loop: 151 for ctx.Err() == nil { 152 tctx, cancel := context.WithTimeout(ctx, statTimeout) 153 g, gctx := errgroup.WithContext(tctx) 154 var ( 155 mem bigmachine.MemInfo 156 merr error 157 disk bigmachine.DiskInfo 158 derr error 159 load bigmachine.LoadInfo 160 lerr error 161 vals stats.Values 162 verr error 163 ) 164 g.Go(func() error { 165 mem, merr = s.Machine.MemInfo(gctx, false) 166 return nil 167 }) 168 g.Go(func() error { 169 disk, derr = s.Machine.DiskInfo(gctx) 170 return nil 171 }) 172 g.Go(func() error { 173 load, lerr = s.Machine.LoadInfo(gctx) 174 return nil 175 }) 176 g.Go(func() error { 177 verr = s.Machine.Call(ctx, "Worker.Stats", struct{}{}, &vals) 178 return nil 179 }) 180 _ = g.Wait() 181 cancel() 182 if merr != nil { 183 log.Debug.Printf("meminfo %s: %v", s.Machine.Addr, merr) 184 } 185 if derr != nil { 186 log.Debug.Printf("diskinfo %s: %v", s.Machine.Addr, derr) 187 } 188 if lerr != nil { 189 log.Debug.Printf("loadinfo %s: %v", s.Machine.Addr, lerr) 190 } 191 if verr != nil { 192 log.Debug.Printf("stats %s: %v", s.Machine.Addr, verr) 193 } 194 s.mu.Lock() 195 if merr == nil { 196 s.mem = mem 197 } 198 if derr == nil { 199 s.disk = disk 200 } 201 if lerr == nil { 202 s.load = load 203 } 204 if verr == nil { 205 s.vals = vals 206 } 207 s.mu.Unlock() 208 s.UpdateStatus() 209 select { 210 case <-time.After(statsPollInterval): 211 case <-ctx.Done(): 212 case <-stopped: 213 break loop 214 } 215 } 216 // The machine is dead: mark it as such and also mark all of its pending 217 // and completed tasks as lost. 218 s.mu.Lock() 219 s.lost = true 220 tasks := s.tasks 221 s.tasks = nil 222 s.mu.Unlock() 223 log.Error.Printf("lost machine %s: marking its %d tasks as LOST", s.Machine.Addr, len(tasks)) 224 for task := range tasks { 225 task.Set(TaskLost) 226 } 227 } 228 229 // Lost reports whether this machine is considered lost. 230 func (s *sliceMachine) Lost() bool { 231 s.mu.Lock() 232 lost := s.lost 233 s.mu.Unlock() 234 return lost 235 } 236 237 // UpdateStatus updates the machine's status. 238 func (s *sliceMachine) UpdateStatus() { 239 s.mu.Lock() 240 defer s.mu.Unlock() 241 values := s.vals.Copy() 242 s.Stats.AddAll(values) 243 var health string 244 switch s.health { 245 case machineOk: 246 case machineProbation: 247 health = " (probation)" 248 case machineLost: 249 health = " (lost)" 250 } 251 s.Status.Printf("mem %s/%s disk %s/%s load %.1f/%.1f/%.1f counters %s%s", 252 data.Size(s.mem.System.Used), data.Size(s.mem.System.Total), 253 data.Size(s.disk.Usage.Used), data.Size(s.disk.Usage.Total), 254 s.load.Averages.Load1, s.load.Averages.Load5, s.load.Averages.Load15, 255 values, health, 256 ) 257 } 258 259 // Load returns the machine's load, i.e., the proportion of its 260 // capacity that is currently in use. 261 func (s *sliceMachine) Load() float64 { 262 return float64(s.taskProcs) / float64(s.maxTaskProcs) 263 } 264 265 // machineFailureQ is a priority queue for sliceMachines, prioritized by the 266 // machine's last failure time, as defined by (*sliceMachine).LastFailure. 267 type machineFailureQ []*sliceMachine 268 269 func (h machineFailureQ) Len() int { return len(h) } 270 func (h machineFailureQ) Less(i, j int) bool { return h[i].lastFailure.Before(h[j].lastFailure) } 271 func (h machineFailureQ) Swap(i, j int) { 272 h[i], h[j] = h[j], h[i] 273 h[i].index, h[j].index = i, j 274 } 275 276 func (h *machineFailureQ) Push(x interface{}) { 277 m := x.(*sliceMachine) 278 m.index = len(*h) 279 *h = append(*h, m) 280 } 281 282 func (h *machineFailureQ) Pop() interface{} { 283 old := *h 284 n := len(old) 285 x := old[n-1] 286 *h = old[0 : n-1] 287 x.index = -1 288 return x 289 } 290 291 // timer is a wrapper around time.Timer with an API convenient for managing 292 // probation timeouts. 293 type timer struct { 294 // t is the underlying *time.Timer. It may be nil. 295 t *time.Timer 296 // at is the time (or later) at which t expired or will expire, if t is 297 // non-nil. 298 at time.Time 299 } 300 301 // Clear clears t; subsequent calls to C() will return nil. If t is already 302 // cleared, no-op. 303 func (t *timer) Clear() { 304 if t.t == nil { 305 return 306 } 307 t.t.Stop() 308 t.t = nil 309 } 310 311 // Set sets t to expire at at. If the timer was already set to expire at at, 312 // no-op, even if the timer has already expired. 313 func (t *timer) Set(at time.Time) { 314 if t.t == nil { 315 t.at = at 316 t.t = time.NewTimer(time.Until(at)) 317 return 318 } 319 if t.at == at { 320 return 321 } 322 if !t.t.Stop() { 323 <-t.t.C 324 } 325 t.at = at 326 t.t.Reset(time.Until(at)) 327 } 328 329 // C returns a channel on which the current time is sent when t expires. If t is 330 // cleared, returns nil. 331 func (t *timer) C() <-chan time.Time { 332 if t.t == nil { 333 return nil 334 } 335 return t.t.C 336 } 337 338 // MachineDone is used to report that a machine's request is done, along 339 // with an error used to gauge the machine's health. 340 type machineDone struct { 341 *sliceMachine 342 // procs is the number of procs to be returned to the pool available for 343 // task assignment on the machine. 344 procs int 345 Err error 346 } 347 348 // startResult is used to signal the result of attempts to start machines. 349 type startResult struct { 350 // machines is a slice of the machines that were successfully started. 351 machines []*sliceMachine 352 // nFailures is the number of machines that we attempted but failed to 353 // start. 354 nFailures int 355 } 356 357 // MachineManager manages a cluster of sliceMachines, load balancing requests 358 // among them. MachineManagers are constructed newMachineManager. 359 type machineManager struct { 360 b *bigmachine.B 361 params []bigmachine.Param 362 group *status.Group 363 maxp int 364 // machprocs is the number of procs each managed machine has available for 365 // tasks, taking into account max load. 366 machprocs int 367 worker *worker 368 // schedQ is the priority queue of scheduling requests, which determines the 369 // order in which requests are satisfied. See Offer. 370 schedQ scheduleRequestQ 371 schedc chan *scheduleRequest 372 unschedc chan *scheduleRequest 373 } 374 375 // NewMachineManager returns a new machineManager paramterized by the 376 // provided arguments. Maxp determines the maximum number of procs 377 // that may be allocated, maxLoad determines the maximum fraction of 378 // machine procs that may be allocated to user work. 379 // 380 // The cluster is not managed until machineManager.Do is called by the user. 381 func newMachineManager(b *bigmachine.B, params []bigmachine.Param, group *status.Group, maxp int, maxLoad float64, worker *worker) *machineManager { 382 // Adjust maxLoad so that we are guaranteed at least one proc per 383 // machine; otherwise we can get stuck in nasty deadlocks. We also 384 // adjust maxp in this case to account for the fact, when maxLoad=0, 385 // we should allocate the entirety of the machine towards a task 386 // with internal parallelism, and the maxp should count towards 387 // that. 388 // 389 // TODO(marius): maxp is still applied on a per-manager basis. It 390 // should be shared across all managers, though this complicates 391 // matters because, without de-allocating machines from one cluster 392 // to another, or at least draining them and transferring them, we 393 // could run into deadlocks. We should probably re-think cluster 394 // management to better accommodate for this. 395 maxprocs := b.System().Maxprocs() 396 machprocs := int(float64(maxprocs) * maxLoad) 397 if machprocs < 1 { 398 machprocs = 1 399 maxp = (maxp + maxprocs - 1) / maxprocs 400 } 401 return &machineManager{ 402 b: b, 403 params: params, 404 group: group, 405 maxp: maxp, 406 machprocs: machprocs, 407 worker: worker, 408 schedc: make(chan *scheduleRequest), 409 unschedc: make(chan *scheduleRequest), 410 } 411 } 412 413 // Offer asks m to offer a machine on which to run work with the given priority 414 // and number of procs. When m schedules the request, the machine is sent to the 415 // returned channel. The second return value is a function that cancels the 416 // request when called. If the request has already been serviced (i.e. a machine 417 // has already been delivered), calling the cancel function is a no-op. 418 func (m *machineManager) Offer(priority, procs int) (<-chan *sliceMachine, func()) { 419 if procs <= 0 { 420 panic("requested procs <= 0") 421 } 422 machc := make(chan *sliceMachine) 423 s := scheduleRequest{ 424 procs: procs, 425 priority: priority, 426 machc: machc, 427 } 428 m.schedc <- &s 429 cancel := func() { 430 m.unschedc <- &s 431 } 432 return machc, cancel 433 } 434 435 // Do starts machine management. The user typically calls this 436 // asynchronously. Do services requests for machine capacity and 437 // monitors machine health: stopped machines are considered lost and 438 // removed from management. 439 // 440 // Do attempts to maintain at least as many procs as are currently 441 // needed (as indicated by client's calls to Need); thus when a 442 // machine is lost, it may be replaced with another should it be 443 // needed. 444 func (m *machineManager) Do(ctx context.Context) { 445 var ( 446 need, pending int 447 startc = make(chan startResult) 448 stoppedc = make(chan *sliceMachine) 449 // numStopped is the total number of machines that have stopped in the 450 // cluster. 451 numStopped int 452 donec = make(chan machineDone) 453 machQ machineQ 454 probation machineFailureQ 455 probationTimer timer 456 // We track consecutive failures to start machines as a heuristic to 457 // decide that there might be a systematic problem preventing machines 458 // from starting. 459 consecutiveStartFailures int 460 logTicker = time.NewTicker(1 * time.Minute) 461 ) 462 defer logTicker.Stop() 463 for { 464 var ( 465 req *scheduleRequest 466 mach *sliceMachine 467 machc chan<- *sliceMachine 468 ) 469 req, mach = schedule(&m.schedQ, &machQ) 470 if req != nil { 471 machc = req.machc 472 } 473 if len(probation) == 0 { 474 probationTimer.Clear() 475 } else { 476 probationTimer.Set(probation[0].lastFailure.Add(ProbationTimeout)) 477 } 478 select { 479 case machc <- mach: 480 mach.taskProcs += req.procs 481 heap.Fix(&machQ, mach.index) 482 heap.Remove(&m.schedQ, req.index) 483 case <-probationTimer.C(): 484 mach := probation[0] 485 mach.health = machineOk 486 log.Printf("removing machine %s from probation", mach.Addr) 487 heap.Remove(&probation, 0) 488 heap.Push(&machQ, mach) 489 probationTimer.Clear() 490 case done := <-donec: 491 need -= done.procs 492 mach := done.sliceMachine 493 mach.taskProcs -= done.procs 494 switch { 495 case done.Err != nil && !errors.Is(errors.Remote, done.Err) && mach.health == machineOk: 496 // We only consider probation if we have problems with RPC 497 // machinery, e.g. host unavailable or other network errors. If 498 // the error is from application code of an RPC, we defer to the 499 // evaluation engine for remediation. This is to limit the blast 500 // radius of a problematic machine, e.g. a call to machine A 501 // transitively calls machine B, but machine B is down; the call 502 // to machine A will return an error, but we do not want to put 503 // machine A on probation. 504 log.Error.Printf("putting machine %s on probation after error: %v", mach, done.Err) 505 mach.health = machineProbation 506 heap.Remove(&machQ, mach.index) 507 mach.lastFailure = time.Now() 508 heap.Push(&probation, mach) 509 case done.Err == nil && mach.health == machineProbation: 510 log.Printf("machine %s returned successful result; removing probation", mach) 511 mach.health = machineOk 512 heap.Remove(&probation, mach.index) 513 heap.Push(&machQ, mach) 514 case mach.health == machineLost: 515 // In this case, the machine has already been removed from the heap. 516 case mach.health == machineProbation: 517 log.Error.Printf("keeping machine %s on probation after error: %v", mach, done.Err) 518 mach.lastFailure = time.Now() 519 heap.Fix(&probation, mach.index) 520 case mach.health == machineOk: 521 // Everything continues merrily. 522 heap.Fix(&machQ, mach.index) 523 default: 524 panic("invalid machine state") 525 } 526 case s := <-m.schedc: 527 heap.Push(&m.schedQ, s) 528 need += s.procs 529 case s := <-m.unschedc: 530 if s.index < 0 { 531 // The scheduling request is no longer queued, which means 532 // scheduling request has already been serviced. 533 break 534 } 535 need -= s.procs 536 heap.Remove(&m.schedQ, s.index) 537 case result := <-startc: 538 pending -= m.machprocs * (len(result.machines) + result.nFailures) 539 for _, mach := range result.machines { 540 heap.Push(&machQ, mach) 541 mach.donec = donec 542 go func(mach *sliceMachine) { 543 <-mach.Wait(bigmachine.Stopped) 544 stoppedc <- mach 545 }(mach) 546 } 547 if len(result.machines) > 0 { 548 consecutiveStartFailures = 0 549 } else { 550 consecutiveStartFailures += result.nFailures 551 if consecutiveStartFailures > 8 { 552 log.Printf("warning; failed to start last %d machines; check for systematic problem preventing machine bootup", consecutiveStartFailures) 553 } 554 } 555 case mach := <-stoppedc: 556 numStopped++ 557 // Remove the machine from management. We let the sliceMachine 558 // instance deal with failing the tasks. 559 log.Error.Printf("machine %s stopped with error %s", mach, mach.Err()) 560 switch mach.health { 561 case machineOk: 562 heap.Remove(&machQ, mach.index) 563 case machineProbation: 564 heap.Remove(&probation, mach.index) 565 } 566 mach.health = machineLost 567 mach.Status.Done() 568 case <-logTicker.C: 569 // pending is in procs, so we convert it to machines. 570 machPending := pending / m.machprocs 571 if len(probation) > 0 { 572 log.Printf("slicemachine: pending/running(probation)/lost: %d/%d(%d)/%d", 573 machPending, len(machQ), len(probation), numStopped) 574 continue 575 } 576 log.Printf("slicemachine: pending/running/lost: %d/%d/%d", 577 machPending, len(machQ), numStopped) 578 continue 579 case <-ctx.Done(): 580 return 581 } 582 583 // TODO(marius): consider scaling down when we don't need as many 584 // resources any more; this would involve moving results to other 585 // machines or to another storage medium. 586 if have := (len(machQ) + len(probation)) * m.machprocs; have+pending < need && have+pending < m.maxp { 587 var ( 588 needProcs = min(need, m.maxp) - have - pending 589 needMachines = min((needProcs+m.machprocs-1)/m.machprocs, maxStartMachines) 590 ) 591 pending += needMachines * m.machprocs 592 log.Printf("slicemachine: %d machines (%d procs); %d machines pending (%d procs)", 593 have/m.machprocs, have, pending/m.machprocs, pending) 594 go func() { 595 started := startMachines(ctx, m.b, m.group, m.machprocs, needMachines, m.worker, m.params...) 596 startc <- startResult{ 597 machines: started, 598 nFailures: needMachines - len(started), 599 } 600 }() 601 } 602 } 603 } 604 605 // schedule attempts to schedule a request from schedQ to run on a machine in 606 // machQ, returning the successfully scheduled request and the machine that 607 // satisfies the request. If no request can be scheduled, returns (nil, nil). 608 // 609 // The ordering of elements of schedQ and machQ may change as a result of 610 // calling schedule. 611 // 612 // It implements the following algorithm: 613 // - Attempt to schedule the highest priority schedule request on the 614 // least-loaded machine. 615 // - If it cannot be scheduled, reserve the machine for that request, 616 // eliminating the machine and request from further consideration. 617 // - Repeat while there are requests to schedule and machines on which they 618 // might possibly be scheduled. 619 // 620 // This works reasonably well for many workloads, however there are degenerate 621 // cases for which this will behave poorly. 622 // 623 // For example, suppose we have an initial batch of small tasks that occupy a 624 // small portion of each machine, followed by a single task that occupies an 625 // entire machine. We'll schedule the small tasks across the machines, and the 626 // full-machine task will not be able to run until one of the machines finishes 627 // the small tasks scheduled on it. If we more densely packed the small tasks, we 628 // would have been able to run the large task on one of the remaining machines. 629 func schedule(schedQ *scheduleRequestQ, machQ *machineQ) (*scheduleRequest, *sliceMachine) { 630 // We may not be able to schedule the highest priority requests. If we 631 // can't, we shelve the requests and look further down the priority queue. 632 // We also shelve (the least loaded) machines to drain them for future 633 // scheduling. 634 var ( 635 shelvedRequests []*scheduleRequest 636 shelvedMachines []*sliceMachine 637 ) 638 defer func() { 639 for i := range shelvedRequests { 640 heap.Push(schedQ, shelvedRequests[i]) 641 heap.Push(machQ, shelvedMachines[i]) 642 } 643 }() 644 for len(*schedQ) > 0 && len(*machQ) > 0 { 645 freeProcs := (*machQ)[0].maxTaskProcs - (*machQ)[0].taskProcs 646 if freeProcs == 0 { 647 // The least-loaded machine has no free procs, so no subsequent 648 // machine in the priority queue will have free procs, so we will 649 // not be able to schedule anything. 650 return nil, nil 651 } 652 if (*schedQ)[0].procs <= freeProcs { 653 return (*schedQ)[0], (*machQ)[0] 654 } 655 shelvedRequests = append(shelvedRequests, heap.Pop(schedQ).(*scheduleRequest)) 656 shelvedMachines = append(shelvedMachines, heap.Pop(machQ).(*sliceMachine)) 657 } 658 return nil, nil 659 } 660 661 // StartMachines starts a number of machines on b, installing a worker service 662 // on each of them. StartMachines returns a slice of successfully started 663 // machines when all of them are in bigmachine.Running state. If a machine 664 // fails to start, it is not included. 665 func startMachines(ctx context.Context, b *bigmachine.B, group *status.Group, maxTaskProcs int, n int, worker *worker, params ...bigmachine.Param) []*sliceMachine { 666 params = append([]bigmachine.Param{bigmachine.Services{"Worker": worker}}, params...) 667 machines, err := b.Start(ctx, n, params...) 668 if err != nil { 669 log.Error.Printf("error starting machines: %v", err) 670 return nil 671 } 672 var wg sync.WaitGroup 673 slicemachines := make([]*sliceMachine, len(machines)) 674 for i := range machines { 675 i := i 676 m := machines[i] 677 status := group.Start() 678 status.Print("waiting for machine to boot") 679 wg.Add(1) 680 go func() { 681 defer wg.Done() 682 <-m.Wait(bigmachine.Running) 683 if err := m.Err(); err != nil { 684 log.Printf("machine %s failed to start: %v", m.Addr, err) 685 status.Printf("failed to start: %v", err) 686 status.Done() 687 return 688 } 689 var workerFuncLocs []string 690 if err := m.RetryCall(ctx, "Worker.FuncLocations", struct{}{}, &workerFuncLocs); err != nil { 691 status.Printf("failed to verify funcs") 692 status.Done() 693 m.Cancel() 694 return 695 } 696 diff := bigslice.FuncLocationsDiff(bigslice.FuncLocations(), workerFuncLocs) 697 if len(diff) > 0 { 698 for _, edit := range diff { 699 log.Printf("[funcsdiff] %s", edit) 700 } 701 log.Panicf("machine %s has different funcs; check for local or non-deterministic Func creation", m.Addr) 702 } 703 status.Title(m.Addr) 704 status.Print("running") 705 log.Printf("machine %v is ready", m.Addr) 706 sm := &sliceMachine{ 707 Machine: m, 708 Stats: stats.NewMap(), 709 Status: status, 710 maxTaskProcs: maxTaskProcs, 711 tasks: make(map[*Task]struct{}), 712 } 713 // TODO(marius): pass a context that's tied to the evaluation 714 // lifetime, or lifetime of the machine. 715 go sm.Go(backgroundcontext.Get()) 716 slicemachines[i] = sm 717 }() 718 } 719 wg.Wait() 720 n = 0 721 for _, m := range slicemachines { 722 if m != nil { 723 slicemachines[n] = m 724 n++ 725 } 726 } 727 return slicemachines[:n] 728 } 729 730 type scheduleRequest struct { 731 // priority is the priority of the request. Lower values have higher 732 // priority. If there is more than one request waiting for a machine, the 733 // request with the lowest priority value will be satisfied first. 734 priority int 735 // procs is the number of procs being requested. 736 procs int 737 machc chan *sliceMachine 738 // index is the index of this request in the request heap. 739 index int 740 } 741 742 // scheduleRequestQ is a priority queue based on request priority and proc 743 // demand. 744 type scheduleRequestQ []*scheduleRequest 745 746 func (q scheduleRequestQ) Len() int { return len(q) } 747 748 func (q scheduleRequestQ) Less(i, j int) bool { 749 if q[i].priority != q[j].priority { 750 return q[i].priority < q[j].priority 751 } 752 // Higher proc demand comes first, as we implement first fit decreasing 753 // scheduling. 754 return q[i].procs > q[j].procs 755 } 756 757 func (q scheduleRequestQ) Swap(i, j int) { 758 q[i], q[j] = q[j], q[i] 759 q[i].index = i 760 q[j].index = j 761 } 762 763 func (q *scheduleRequestQ) Push(x interface{}) { 764 n := len(*q) 765 s := x.(*scheduleRequest) 766 s.index = n 767 *q = append(*q, s) 768 } 769 770 func (q *scheduleRequestQ) Pop() interface{} { 771 old := *q 772 n := len(old) 773 s := old[n-1] 774 s.index = -1 775 *q = old[:n-1] 776 return s 777 } 778 779 // machineQ is a priority queue for sliceMachines, prioritized by the machine's 780 // load (by procs). We use this to hold our active machines, as we generally 781 // schedule tasks to run on the least-loaded machines first. See schedule. 782 type machineQ []*sliceMachine 783 784 func (q machineQ) Len() int { return len(q) } 785 786 func (q machineQ) Less(i, j int) bool { 787 return q[j].maxTaskProcs-q[j].taskProcs < q[i].maxTaskProcs-q[i].taskProcs 788 } 789 790 func (q machineQ) Swap(i, j int) { 791 q[i], q[j] = q[j], q[i] 792 q[i].index = i 793 q[j].index = j 794 } 795 796 func (q *machineQ) Push(x interface{}) { 797 n := len(*q) 798 s := x.(*sliceMachine) 799 s.index = n 800 *q = append(*q, s) 801 } 802 803 func (q *machineQ) Pop() interface{} { 804 old := *q 805 n := len(old) 806 m := old[n-1] 807 m.index = -1 808 *q = old[:n-1] 809 return m 810 } 811 812 func min(x, y int) int { 813 if x < y { 814 return x 815 } 816 return y 817 }