github.com/grailbio/bigslice@v0.0.0-20230519005545-30c4c12152ad/exec/slicestatus.go (about) 1 // Copyright 2019 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 "context" 9 "fmt" 10 "time" 11 12 "github.com/grailbio/base/log" 13 "github.com/grailbio/base/status" 14 "github.com/grailbio/bigslice" 15 ) 16 17 // sliceStatus is the information directly used to print a slice's status (in a 18 // *status.Task). 19 type sliceStatus struct { 20 // sliceName is the name of the slice to which this status information 21 // applies. 22 sliceName bigslice.Name 23 // counts is the count of tasks of the slice, by TaskState. 24 counts [maxState]int32 25 } 26 27 // printTo prints s to t, translating our slice status information to a 28 // status.Task update. 29 func (s sliceStatus) printTo(t *status.Task) { 30 t.Print(taskCountsString(s.counts)) 31 } 32 33 // taskCountsString returns a human-consumable string representing counts which 34 // holds the counts of tasks per state. 35 func taskCountsString(counts [maxState]int32) string { 36 idleCount := counts[TaskInit] + counts[TaskWaiting] 37 if counts[TaskLost] > 0 || counts[TaskErr] > 0 { 38 return fmt.Sprintf("tasks idle/running/done(lost)/error: %d/%d/%d(%d)/%d", 39 idleCount, counts[TaskRunning], counts[TaskOk], 40 counts[TaskLost], counts[TaskErr]) 41 } 42 return fmt.Sprintf("tasks idle/running/done: %d/%d/%d", idleCount, 43 counts[TaskRunning], counts[TaskOk]) 44 } 45 46 // iterTasks calls f for each task in the full graph specified by tasks. It is 47 // post-order DFS so that tasks are visited in a valid execution order. We use 48 // this property to display slice status in a sensible order. If f returns an 49 // error, iterTasks halts and returns that error. 50 func iterTasks(tasks []*Task, f func(*Task) error) error { 51 visited := make(map[*Task]struct{}) 52 var walk func([]*Task) error 53 walk = func(tasks []*Task) error { 54 if len(tasks) == 0 { 55 return nil 56 } 57 // This optimization to only use the first task as a marker for 58 // visitation is safe because task slices that result from compilation 59 // are either identical or mutually exclusive. 60 if _, ok := visited[tasks[0]]; ok { 61 return nil 62 } 63 visited[tasks[0]] = struct{}{} 64 for _, t := range tasks { 65 for _, d := range t.Deps { 66 for i := 0; i < d.NumTask(); i++ { 67 if err := walk([]*Task{d.Task(i)}); err != nil { 68 return err 69 } 70 } 71 } 72 if err := f(t); err != nil { 73 return err 74 } 75 } 76 return nil 77 } 78 return walk(tasks) 79 } 80 81 // maintainSliceGroup maintains a status.Group that tracks the evaluation status 82 // of the slices computed by tasks. This is usually called in a goroutine and 83 // returns only when ctx is done. 84 func maintainSliceGroup(ctx context.Context, tasks []*Task, group *status.Group) { 85 sliceToStatusTask := make(map[bigslice.Name]*status.Task) 86 // We set up a status.Task for each slice computed by the given task graph. 87 _ = iterTasks(tasks, func(t *Task) error { 88 for i := len(t.Slices) - 1; i >= 0; i-- { 89 // The slices are in dependency order, so we visit them in reverse 90 // to get them in execution order. 91 s := t.Slices[i] 92 if _, ok := sliceToStatusTask[s.Name()]; !ok { 93 sliceToStatusTask[s.Name()] = group.Start(s.Name().String()) 94 } 95 } 96 return nil 97 }) 98 group.Printf("count: %d", len(sliceToStatusTask)) 99 statusc := make(chan sliceStatus) 100 go monitorSliceStatus(ctx, tasks, statusc) 101 for status := range statusc { 102 status.printTo(sliceToStatusTask[status.sliceName]) 103 } 104 for _, statusTask := range sliceToStatusTask { 105 statusTask.Printf("tasks done") 106 statusTask.Done() 107 } 108 group.Printf("count: %d; done", len(sliceToStatusTask)) 109 } 110 111 // monitorSliceStatus continually sends sliceStatus to statusc as the states of 112 // tasks are updated. It will only return only when ctx is done. 113 func monitorSliceStatus(ctx context.Context, tasks []*Task, statusc chan<- sliceStatus) { 114 sub := NewTaskSubscriber() 115 taskToLastState := make(map[*Task]TaskState) 116 sliceToStatus := make(map[bigslice.Name]sliceStatus) 117 _ = iterTasks(tasks, func(t *Task) error { 118 // Subscribe to updates before we grab the initial state so that we 119 // are guaranteed to see every subsequent update. 120 t.Subscribe(sub) 121 taskState := t.State() 122 taskToLastState[t] = taskState 123 for _, s := range t.Slices { 124 status := sliceToStatus[s.Name()] 125 status.sliceName = s.Name() 126 status.counts[taskState]++ 127 sliceToStatus[s.Name()] = status 128 statusc <- status 129 } 130 return nil 131 }) 132 defer func() { 133 _ = iterTasks(tasks, func(t *Task) error { 134 t.Unsubscribe(sub) 135 return nil 136 }) 137 }() 138 // Initial state is ready. Observe updates. 139 for { 140 select { 141 case <-sub.Ready(): 142 for _, task := range sub.Tasks() { 143 lastState := taskToLastState[task] 144 state := task.State() 145 for _, s := range task.Slices { 146 status := sliceToStatus[s.Name()] 147 status.counts[lastState]-- 148 status.counts[state]++ 149 sliceToStatus[s.Name()] = status 150 statusc <- status 151 } 152 taskToLastState[task] = state 153 } 154 case <-ctx.Done(): 155 close(statusc) 156 return 157 } 158 } 159 } 160 161 // logInvocation periodically logs about the state of a given invocation inv 162 // and its tasks. 163 func logInvocation(ctx context.Context, inv execInvocation, tasks []*Task) { 164 var ( 165 sub = NewTaskSubscriber() 166 taskToLastState = make(map[*Task]TaskState) 167 counts [maxState]int32 168 ) 169 _ = iterTasks(tasks, func(t *Task) error { 170 // Subscribe to updates before we grab the initial state so that we 171 // are guaranteed to see every subsequent update. 172 t.Subscribe(sub) 173 taskState := t.State() 174 taskToLastState[t] = taskState 175 counts[taskState]++ 176 return nil 177 }) 178 defer func() { 179 _ = iterTasks(tasks, func(t *Task) error { 180 t.Unsubscribe(sub) 181 return nil 182 }) 183 }() 184 t := time.NewTicker(1 * time.Minute) 185 defer t.Stop() 186 for { 187 select { 188 case <-sub.Ready(): 189 for _, task := range sub.Tasks() { 190 lastState := taskToLastState[task] 191 counts[lastState]-- 192 state := task.State() 193 taskToLastState[task] = state 194 counts[state]++ 195 } 196 case <-t.C: 197 log.Printf("invocation: %s(%d): %s", inv.Location, inv.Index, taskCountsString(counts)) 198 case <-ctx.Done(): 199 return 200 } 201 } 202 }