github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/executor/worker/task_runner.go (about) 1 // Copyright 2022 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package worker 15 16 import ( 17 "context" 18 "sync" 19 20 "github.com/pingcap/log" 21 "github.com/pingcap/tiflow/engine/executor/worker/internal" 22 "github.com/pingcap/tiflow/engine/pkg/clock" 23 "github.com/pingcap/tiflow/engine/pkg/notifier" 24 "github.com/pingcap/tiflow/pkg/errors" 25 "go.uber.org/atomic" 26 "go.uber.org/zap" 27 ) 28 29 // Re-export types for public use 30 type ( 31 // Runnable alias internal.Runnable 32 Runnable = internal.Runnable 33 // RunnableID alias internal.RunnableID 34 RunnableID = internal.RunnableID 35 ) 36 37 // TaskRunner receives RunnableContainer in a FIFO way, and runs them in 38 // independent background goroutines. 39 type TaskRunner struct { 40 inQueue chan *internal.RunnableContainer 41 tasks sync.Map 42 wg sync.WaitGroup 43 44 cancelMu sync.RWMutex 45 canceled bool 46 47 taskCount atomic.Int64 48 49 clock clock.Clock 50 51 taskStopNotifier *notifier.Notifier[RunnableID] 52 } 53 54 type taskEntry struct { 55 *internal.RunnableContainer 56 cancel context.CancelFunc 57 } 58 59 // NewTaskRunner creates a new TaskRunner instance 60 func NewTaskRunner(inQueueSize int, initConcurrency int) *TaskRunner { 61 return &TaskRunner{ 62 inQueue: make(chan *internal.RunnableContainer, inQueueSize), 63 clock: clock.New(), 64 taskStopNotifier: notifier.NewNotifier[RunnableID](), 65 } 66 } 67 68 // AddTask enqueues a naked task, and AddTask will wrap the task with internal.WrapRunnable. 69 // Deprecated. TODO Will be removed once two-phase task dispatching is enabled. 70 func (r *TaskRunner) AddTask(task Runnable) error { 71 wrappedTask := internal.WrapRunnable(task, r.clock.Mono()) 72 select { 73 case r.inQueue <- wrappedTask: 74 return nil 75 default: 76 } 77 78 return errors.ErrRuntimeIncomingQueueFull.GenWithStackByArgs() 79 } 80 81 // addWrappedTask enqueues a task already wrapped by internal.WrapRunnable. 82 // NOTE: internal.RunnableContainer contains the submit-time for the task. 83 func (r *TaskRunner) addWrappedTask(task *internal.RunnableContainer) error { 84 select { 85 case r.inQueue <- task: 86 return nil 87 default: 88 } 89 90 return errors.ErrRuntimeIncomingQueueFull.GenWithStackByArgs() 91 } 92 93 // Run runs forever until context is canceled or task queue is closed. 94 // It receives new added task and call onNewTask with task 95 func (r *TaskRunner) Run(ctx context.Context) error { 96 defer r.cancelAll() 97 98 for { 99 select { 100 case <-ctx.Done(): 101 return errors.Trace(ctx.Err()) 102 case task := <-r.inQueue: 103 if task == nil { 104 return errors.ErrRuntimeIsClosed.GenWithStackByArgs() 105 } 106 if err := r.onNewTask(task); err != nil { 107 log.Warn("Failed to launch task", 108 zap.String("id", task.ID()), 109 zap.Error(err)) 110 } 111 } 112 } 113 } 114 115 // WorkerCount returns the number of currently running workers. 116 func (r *TaskRunner) WorkerCount() int64 { 117 return r.taskCount.Load() 118 } 119 120 func (r *TaskRunner) cancelAll() { 121 r.cancelMu.Lock() 122 if r.canceled { 123 return 124 } 125 r.canceled = true 126 127 r.tasks.Range(func(key, value interface{}) bool { 128 id := key.(RunnableID) 129 t := value.(*taskEntry) 130 t.cancel() 131 log.Info("Cancelling task", zap.String("id", id)) 132 return true 133 }) 134 r.cancelMu.Unlock() 135 136 r.taskStopNotifier.Close() 137 r.wg.Wait() 138 } 139 140 func (r *TaskRunner) onNewTask(task *internal.RunnableContainer) (ret error) { 141 defer func() { 142 if r := recover(); r != nil { 143 ret = errors.Trace(errors.Errorf("panic: %v", r)) 144 } 145 }() 146 147 taskCtx, cancel := context.WithCancel(context.Background()) 148 t := &taskEntry{ 149 RunnableContainer: task, 150 cancel: cancel, 151 } 152 153 rctx := newRuntimeCtx(taskCtx, task.Info()) 154 155 r.cancelMu.RLock() 156 defer r.cancelMu.RUnlock() 157 158 if r.canceled { 159 return errors.ErrRuntimeClosed.GenWithStackByArgs() 160 } 161 162 _, exists := r.tasks.LoadOrStore(task.ID(), t) 163 if exists { 164 log.Warn("Duplicate Task ID", zap.String("id", task.ID())) 165 return errors.ErrRuntimeDuplicateTaskID.GenWithStackByArgs(task.ID()) 166 } 167 168 r.launchTask(rctx, t) 169 170 return nil 171 } 172 173 func (r *TaskRunner) launchTask(rctx *RuntimeContext, entry *taskEntry) { 174 r.wg.Add(1) 175 r.taskCount.Inc() 176 177 go func() { 178 defer r.wg.Done() 179 defer r.taskCount.Dec() 180 181 var err error 182 defer func() { 183 if r2 := recover(); r2 != nil { 184 err2 := errors.Trace(errors.Errorf("panic: %v", r2)) 185 log.Error("Task panicked", zap.String("id", entry.ID()), zap.Error(err2)) 186 } 187 log.Info("Task Closed", 188 zap.String("id", entry.ID()), 189 zap.Error(err), 190 zap.Int64("runtime-task-count", r.taskCount.Load())) 191 entry.OnStopped() 192 r.taskStopNotifier.Notify(entry.ID()) 193 if _, ok := r.tasks.LoadAndDelete(entry.ID()); !ok { 194 log.Panic("Task does not exist", zap.String("id", entry.ID())) 195 } 196 }() 197 198 entry.OnLaunched() 199 log.Info("Launching task", 200 zap.String("id", entry.ID()), 201 zap.Int64("runtime-task-count", r.taskCount.Load())) 202 203 err = entry.Run(rctx) 204 log.Info("Task stopped", zap.String("id", entry.ID()), zap.Error(err)) 205 }() 206 } 207 208 // TaskCount returns current task count 209 func (r *TaskRunner) TaskCount() int64 { 210 return r.taskCount.Load() 211 } 212 213 // TaskStopReceiver returns a *notifier.Notifier to notify when task is stopped. 214 func (r *TaskRunner) TaskStopReceiver() *notifier.Receiver[RunnableID] { 215 return r.taskStopNotifier.NewReceiver() 216 }