github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/servermaster/executor_manager.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 servermaster 15 16 import ( 17 "context" 18 "sync" 19 "time" 20 21 "github.com/pingcap/log" 22 pb "github.com/pingcap/tiflow/engine/enginepb" 23 "github.com/pingcap/tiflow/engine/model" 24 "github.com/pingcap/tiflow/engine/pkg/notifier" 25 "github.com/pingcap/tiflow/engine/pkg/orm" 26 ormModel "github.com/pingcap/tiflow/engine/pkg/orm/model" 27 schedModel "github.com/pingcap/tiflow/engine/servermaster/scheduler/model" 28 "github.com/pingcap/tiflow/pkg/errors" 29 "github.com/pingcap/tiflow/pkg/label" 30 "go.uber.org/zap" 31 "golang.org/x/time/rate" 32 ) 33 34 // ExecutorManager defines an interface to manager all executors 35 type ExecutorManager interface { 36 HandleHeartbeat(req *pb.HeartbeatRequest) (*pb.HeartbeatResponse, error) 37 AllocateNewExec(ctx context.Context, req *pb.RegisterExecutorRequest) (*ormModel.Executor, error) 38 // ExecutorCount returns executor count with given status 39 ExecutorCount(status model.ExecutorStatus) int 40 HasExecutor(executorID string) bool 41 ListExecutors() []*ormModel.Executor 42 GetAddr(executorID model.ExecutorID) (string, bool) 43 Run(ctx context.Context) error 44 45 // WatchExecutors returns a snapshot of all online executors plus 46 // a stream of events describing changes that happen to the executors 47 // after the snapshot is taken. 48 WatchExecutors(ctx context.Context) ( 49 snap map[model.ExecutorID]string, stream *notifier.Receiver[model.ExecutorStatusChange], err error, 50 ) 51 52 // GetExecutorInfos implements the interface scheduler.executorInfoProvider. 53 // It is called by the scheduler as the source of truth for executors. 54 GetExecutorInfos() map[model.ExecutorID]schedModel.ExecutorInfo 55 } 56 57 // ExecutorManagerImpl holds all the executors' info, including liveness, status, resource usage. 58 type ExecutorManagerImpl struct { 59 wg sync.WaitGroup 60 metaClient orm.Client 61 62 mu sync.Mutex 63 executors map[model.ExecutorID]*Executor 64 65 initHeartbeatTTL time.Duration 66 keepAliveInterval time.Duration 67 68 logRL *rate.Limiter 69 70 notifier *notifier.Notifier[model.ExecutorStatusChange] 71 } 72 73 // NewExecutorManagerImpl creates a new ExecutorManagerImpl instance 74 func NewExecutorManagerImpl(metaClient orm.Client, initHeartbeatTTL, keepAliveInterval time.Duration) *ExecutorManagerImpl { 75 return &ExecutorManagerImpl{ 76 metaClient: metaClient, 77 executors: make(map[model.ExecutorID]*Executor), 78 initHeartbeatTTL: initHeartbeatTTL, 79 keepAliveInterval: keepAliveInterval, 80 logRL: rate.NewLimiter(rate.Every(time.Second*5), 1 /*burst*/), 81 notifier: notifier.NewNotifier[model.ExecutorStatusChange](), 82 } 83 } 84 85 // removeExecutorLocked removes an executor from the manager. 86 // Note that this method must be called with the lock held. 87 func (e *ExecutorManagerImpl) removeExecutorLocked(id model.ExecutorID) error { 88 log.Info("begin to remove executor", zap.String("id", string(id))) 89 exec, ok := e.executors[id] 90 if !ok { 91 // This executor has been removed 92 return errors.ErrUnknownExecutor.GenWithStackByArgs(id) 93 } 94 delete(e.executors, id) 95 log.Info("notify to offline exec") 96 97 e.notifier.Notify(model.ExecutorStatusChange{ 98 ID: id, 99 Tp: model.EventExecutorOffline, 100 Addr: exec.Address, 101 }) 102 103 // Delete the executor from the database. This operation may take a long time, 104 // and it may fail if the database is unavailable. So we don't want to hold the lock. 105 // Instead, we do the deletion in a goroutine. 106 // 107 // We use ttl mechanism to manage the executor's life cycle. So we can tolerate 108 // that a tombstone executor may be left in the database. 109 e.wg.Add(1) 110 go func() { 111 defer e.wg.Done() 112 113 ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) 114 defer cancel() 115 116 if err := e.metaClient.DeleteExecutor(ctx, id); err != nil { 117 log.Warn("failed to delete executor from database", zap.String("id", string(id)), zap.Error(err)) 118 } 119 }() 120 121 return nil 122 } 123 124 // HandleHeartbeat implements pb interface, 125 func (e *ExecutorManagerImpl) HandleHeartbeat(req *pb.HeartbeatRequest) (*pb.HeartbeatResponse, error) { 126 if e.logRL.Allow() { 127 log.Info("handle heart beat", zap.Stringer("req", req)) 128 } 129 e.mu.Lock() 130 execID := model.ExecutorID(req.ExecutorId) 131 exec, ok := e.executors[execID] 132 133 // executor not exists 134 if !ok { 135 e.mu.Unlock() 136 return nil, errors.ErrUnknownExecutor.GenWithStackByArgs(execID) 137 } 138 e.mu.Unlock() 139 140 if err := exec.heartbeat(req.Ttl); err != nil { 141 return nil, err 142 } 143 resp := &pb.HeartbeatResponse{} 144 return resp, nil 145 } 146 147 // registerExec registers executor to both executor manager and resource manager. 148 // Note that this method must be called with the lock held. 149 func (e *ExecutorManagerImpl) registerExecLocked(executorMeta *ormModel.Executor) { 150 log.Info("register executor", zap.Any("executor", executorMeta)) 151 exec := &Executor{ 152 Executor: *executorMeta, 153 lastUpdateTime: time.Now(), 154 heartbeatTTL: e.initHeartbeatTTL, 155 status: model.Initing, 156 logRL: rate.NewLimiter(rate.Every(time.Second*5), 1 /*burst*/), 157 } 158 e.executors[executorMeta.ID] = exec 159 e.notifier.Notify(model.ExecutorStatusChange{ 160 ID: executorMeta.ID, 161 Tp: model.EventExecutorOnline, 162 Addr: executorMeta.Address, 163 }) 164 } 165 166 // AllocateNewExec allocates new executor info to a give RegisterExecutorRequest 167 // and then registers the executor. 168 func (e *ExecutorManagerImpl) AllocateNewExec(ctx context.Context, req *pb.RegisterExecutorRequest) (*ormModel.Executor, error) { 169 pbExecutor := req.Executor 170 log.Info("allocate new executor", zap.Stringer("executor", pbExecutor)) 171 172 e.mu.Lock() 173 var executorID model.ExecutorID 174 for { 175 executorID = model.ExecutorID(generateNodeID(pbExecutor.GetName())) 176 if _, ok := e.executors[executorID]; !ok { 177 break 178 } 179 } 180 181 labelSet, err := label.NewSetFromMap(req.GetExecutor().GetLabels()) 182 if err != nil { 183 return nil, err 184 } 185 executorMeta := &ormModel.Executor{ 186 ID: executorID, 187 Name: pbExecutor.GetName(), 188 Address: pbExecutor.GetAddress(), 189 Labels: ormModel.LabelSet(labelSet), 190 } 191 e.registerExecLocked(executorMeta) 192 e.mu.Unlock() 193 194 // Store the executor info to database. 195 // If any error occurs, client shouldn't use the executor. 196 // The executor in the map will be removed after the ttl expires. 197 if err := e.metaClient.CreateExecutor(ctx, executorMeta); err != nil { 198 return nil, errors.Trace(err) 199 } 200 201 return executorMeta, nil 202 } 203 204 // HasExecutor implements ExecutorManager.HasExecutor 205 func (e *ExecutorManagerImpl) HasExecutor(executorID string) bool { 206 e.mu.Lock() 207 defer e.mu.Unlock() 208 _, ok := e.executors[model.ExecutorID(executorID)] 209 return ok 210 } 211 212 // ListExecutors implements ExecutorManager.ListExecutors 213 func (e *ExecutorManagerImpl) ListExecutors() []*ormModel.Executor { 214 e.mu.Lock() 215 defer e.mu.Unlock() 216 ret := make([]*ormModel.Executor, 0, len(e.executors)) 217 for _, exec := range e.executors { 218 execMeta := exec.Executor 219 ret = append(ret, &execMeta) 220 } 221 return ret 222 } 223 224 // Executor records the status of an executor instance. 225 type Executor struct { 226 ormModel.Executor 227 status model.ExecutorStatus 228 229 mu sync.RWMutex 230 // Last heartbeat 231 lastUpdateTime time.Time 232 heartbeatTTL time.Duration 233 logRL *rate.Limiter 234 } 235 236 func (e *Executor) checkAlive() bool { 237 if e.logRL.Allow() { 238 log.Info("check alive", zap.String("exec", string(e.ID))) 239 } 240 241 e.mu.Lock() 242 defer e.mu.Unlock() 243 if e.status == model.Tombstone { 244 return false 245 } 246 if e.lastUpdateTime.Add(e.heartbeatTTL).Before(time.Now()) { 247 e.status = model.Tombstone 248 return false 249 } 250 return true 251 } 252 253 func (e *Executor) heartbeat(ttl uint64) error { 254 e.mu.Lock() 255 defer e.mu.Unlock() 256 if e.status == model.Tombstone { 257 return errors.ErrTombstoneExecutor.GenWithStackByArgs(e.ID) 258 } 259 e.lastUpdateTime = time.Now() 260 e.heartbeatTTL = time.Duration(ttl) * time.Millisecond 261 e.status = model.Running 262 return nil 263 } 264 265 func (e *Executor) statusEqual(status model.ExecutorStatus) bool { 266 e.mu.RLock() 267 defer e.mu.RUnlock() 268 return e.status == status 269 } 270 271 // Run implements ExecutorManager.Run 272 func (e *ExecutorManagerImpl) Run(ctx context.Context) error { 273 if err := e.resetExecutors(ctx); err != nil { 274 return errors.Errorf("failed to reset executors: %v", err) 275 } 276 277 ticker := time.NewTicker(e.keepAliveInterval) 278 defer func() { 279 ticker.Stop() 280 e.notifier.Close() 281 log.Info("executor manager exited") 282 }() 283 for { 284 select { 285 case <-ctx.Done(): 286 return errors.Trace(ctx.Err()) 287 case <-ticker.C: 288 err := e.checkAliveImpl() 289 if err != nil { 290 log.Info("check alive meet error", zap.Error(err)) 291 } 292 } 293 } 294 } 295 296 func (e *ExecutorManagerImpl) checkAliveImpl() error { 297 e.mu.Lock() 298 defer e.mu.Unlock() 299 300 for id, exec := range e.executors { 301 if !exec.checkAlive() { 302 err := e.removeExecutorLocked(id) 303 return err 304 } 305 } 306 return nil 307 } 308 309 // ExecutorCount implements ExecutorManager.ExecutorCount 310 func (e *ExecutorManagerImpl) ExecutorCount(status model.ExecutorStatus) (count int) { 311 e.mu.Lock() 312 defer e.mu.Unlock() 313 for _, executor := range e.executors { 314 if executor.statusEqual(status) { 315 count++ 316 } 317 } 318 return 319 } 320 321 // GetAddr implements ExecutorManager.GetAddr 322 func (e *ExecutorManagerImpl) GetAddr(executorID model.ExecutorID) (string, bool) { 323 e.mu.Lock() 324 defer e.mu.Unlock() 325 326 executor, exists := e.executors[executorID] 327 if !exists { 328 return "", false 329 } 330 331 return executor.Address, true 332 } 333 334 func (e *ExecutorManagerImpl) resetExecutors(ctx context.Context) error { 335 e.mu.Lock() 336 defer e.mu.Unlock() 337 338 orphanExecutorIDs := make(map[model.ExecutorID]struct{}) 339 for id := range e.executors { 340 orphanExecutorIDs[id] = struct{}{} 341 } 342 343 executors, err := e.metaClient.QueryExecutors(ctx) 344 if err != nil { 345 return errors.Trace(err) 346 } 347 for _, executor := range executors { 348 e.registerExecLocked(executor) 349 delete(orphanExecutorIDs, executor.ID) 350 } 351 352 // Clean up executors that are not in the meta. 353 for id := range orphanExecutorIDs { 354 if err := e.removeExecutorLocked(id); err != nil { 355 return errors.Trace(err) 356 } 357 } 358 359 return nil 360 } 361 362 // WatchExecutors implements the ExecutorManager interface. 363 func (e *ExecutorManagerImpl) WatchExecutors( 364 ctx context.Context, 365 ) (snap map[model.ExecutorID]string, receiver *notifier.Receiver[model.ExecutorStatusChange], err error) { 366 e.mu.Lock() 367 defer e.mu.Unlock() 368 369 snap = make(map[model.ExecutorID]string, len(e.executors)) 370 for executorID, exec := range e.executors { 371 snap[executorID] = exec.Address 372 } 373 374 if err := e.notifier.Flush(ctx); err != nil { 375 return nil, nil, err 376 } 377 378 receiver = e.notifier.NewReceiver() 379 return 380 } 381 382 // GetExecutorInfos returns necessary information on the executor that 383 // is needed for scheduling. 384 func (e *ExecutorManagerImpl) GetExecutorInfos() map[model.ExecutorID]schedModel.ExecutorInfo { 385 e.mu.Lock() 386 defer e.mu.Unlock() 387 388 ret := make(map[model.ExecutorID]schedModel.ExecutorInfo, len(e.executors)) 389 for id, exec := range e.executors { 390 schedInfo := schedModel.ExecutorInfo{ 391 ID: id, 392 Labels: label.Set(exec.Labels), 393 } 394 ret[id] = schedInfo 395 } 396 return ret 397 }