github.com/grailbio/bigslice@v0.0.0-20230519005545-30c4c12152ad/exec/evalstress_test.go (about) 1 // Copyright 2020 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 "math/rand" 10 "net/http" 11 "sync" 12 "testing" 13 "time" 14 15 "github.com/grailbio/base/eventlog" 16 "github.com/grailbio/bigslice" 17 "github.com/grailbio/bigslice/sliceio" 18 ) 19 20 // stressExecutor implements Executor with randomized task loss to stress the 21 // correctness of evaluation. 22 type stressExecutor struct { 23 wg sync.WaitGroup 24 // tasksOk is the set of tasks that this executor has ever successfully 25 // completed. We use this to verify that once Eval returns, all root tasks 26 // have, at least at some point, completed successfully. 27 tasksOk map[*Task]bool 28 // cTasksOk is sent tasks to be recorded in tasksOk. 29 cTasksOk chan *Task 30 // cReadTaskOk is sent requests to read whether a task a task has ever been 31 // completed successfully by this executor. 32 cReadTaskOk chan msgReadTaskOk 33 errorRate float64 34 } 35 36 type msgReadTaskOk struct { 37 task *Task 38 c chan bool 39 } 40 41 func newStressExecutor() *stressExecutor { 42 e := &stressExecutor{ 43 tasksOk: make(map[*Task]bool), 44 cTasksOk: make(chan *Task), 45 cReadTaskOk: make(chan msgReadTaskOk), 46 } 47 return e 48 } 49 50 func (*stressExecutor) Name() string { 51 return "stress" 52 } 53 54 func (e *stressExecutor) Start(*Session) func() { 55 ctx, cancel := context.WithCancel(context.Background()) 56 go e.loop(ctx) 57 return func() { 58 cancel() 59 e.wg.Wait() 60 } 61 } 62 63 // delay delays for a random duration to exercise different sequencing. 64 func delay() { 65 delayMS := time.Duration(rand.Intn(50) + 10) 66 <-time.After(delayMS * time.Millisecond) 67 } 68 69 func (e *stressExecutor) Run(task *Task) { 70 e.wg.Add(1) 71 go func() { 72 defer e.wg.Done() 73 delay() 74 task.Set(TaskRunning) 75 completionState := TaskOk 76 if rand.Float64() < e.errorRate { 77 completionState = TaskLost 78 } 79 delay() 80 if completionState == TaskOk { 81 e.cTasksOk <- task 82 } 83 task.Set(completionState) 84 }() 85 } 86 87 func (*stressExecutor) Reader(*Task, int) sliceio.ReadCloser { 88 panic("not implemented") 89 } 90 91 func (*stressExecutor) Discard(ctx context.Context, task *Task) { 92 panic("not implemented") 93 } 94 95 func (*stressExecutor) Eventer() eventlog.Eventer { 96 return eventlog.Nop{} 97 } 98 99 func (*stressExecutor) HandleDebug(handler *http.ServeMux) { 100 panic("not implemented") 101 } 102 103 func (e *stressExecutor) loop(ctx context.Context) { 104 for { 105 select { 106 case task := <-e.cTasksOk: 107 e.tasksOk[task] = true 108 if rand.Float64() < e.errorRate { 109 // Simulate spontaneous task loss. 110 e.wg.Add(1) 111 go func() { 112 defer e.wg.Done() 113 delay() 114 task.Set(TaskLost) 115 }() 116 } 117 case m := <-e.cReadTaskOk: 118 m.c <- e.tasksOk[m.task] 119 case <-ctx.Done(): 120 return 121 } 122 } 123 } 124 125 func (e *stressExecutor) taskOk(task *Task) bool { 126 c := make(chan bool) 127 e.cReadTaskOk <- msgReadTaskOk{task: task, c: c} 128 return <-c 129 } 130 131 func (e *stressExecutor) setErrorRate(errorRate float64) { 132 e.errorRate = errorRate 133 } 134 135 // TestEvalStress verifies that evaluation behaves properly in the face of task 136 // loss. 137 func TestEvalStress(t *testing.T) { 138 if testing.Short() { 139 t.Skip("skipping test in short mode.") 140 } 141 // Disable handling of consecutive lost tasks so that the evaluator tries 142 // indefinitely. We will be stressing the evaluator with very high error 143 // rates and are verifying its behavior, even though in practice we would 144 // give up, as error rates this high would generally mean some systematic 145 // problem that needs to be fixed. 146 origEnableMaxConsecutiveLost := enableMaxConsecutiveLost 147 enableMaxConsecutiveLost = false 148 defer func() { 149 enableMaxConsecutiveLost = origEnableMaxConsecutiveLost 150 }() 151 var ( 152 executor = newStressExecutor() 153 shutdown = executor.Start(nil) 154 ctx = context.Background() 155 ) 156 for i := 0; i < 10; i++ { 157 tasks, _, _ := compileFunc(func() bigslice.Slice { 158 vs := make([]int, 1000) 159 for i := range vs { 160 vs[i] = i 161 } 162 slice := bigslice.Const(100, vs) 163 slice2 := bigslice.Const(100, vs) 164 slice = bigslice.Cogroup(slice) 165 slice = bigslice.Map(slice, func(i int) int { return i + 1 }) 166 slice = bigslice.Cogroup(slice, slice2) 167 slice = bigslice.Map(slice, func(i int) (int, int) { return i, i + 1 }) 168 slice = bigslice.Reduce(slice, func(a, b int) int { return a + b }) 169 return slice 170 }) 171 executor.setErrorRate(rand.Float64() * 0.2) 172 err := Eval(ctx, executor, tasks, nil) 173 if err != nil { 174 t.Fatalf("unexpected error: %v", err) 175 } 176 // Verify that all root tasks have been computed successfully at some 177 // point. 178 for _, task := range tasks { 179 if !executor.taskOk(task) { 180 t.Errorf("task not evaluated: %v", task) 181 } 182 } 183 } 184 shutdown() 185 }