github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/jobmaster/cvsjob/cvs_job_master.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 cvs 15 16 import ( 17 "context" 18 "encoding/json" 19 "sync" 20 "time" 21 "unsafe" 22 23 "github.com/gin-gonic/gin" 24 "github.com/pingcap/log" 25 cvsTask "github.com/pingcap/tiflow/engine/executor/cvs" 26 "github.com/pingcap/tiflow/engine/executor/worker" 27 "github.com/pingcap/tiflow/engine/framework" 28 frameModel "github.com/pingcap/tiflow/engine/framework/model" 29 "github.com/pingcap/tiflow/engine/framework/registry" 30 "github.com/pingcap/tiflow/engine/pkg/clock" 31 dcontext "github.com/pingcap/tiflow/engine/pkg/context" 32 "github.com/pingcap/tiflow/engine/pkg/p2p" 33 "github.com/pingcap/tiflow/pkg/errors" 34 "go.uber.org/atomic" 35 "go.uber.org/zap" 36 "golang.org/x/time/rate" 37 ) 38 39 // Config records all configurations of cvs job 40 type Config struct { 41 SrcHost string `toml:"srcHost" json:"srcHost"` 42 SrcDir string `toml:"srcDir" json:"srcDir"` 43 DstHost string `toml:"dstHost" json:"dstHost"` 44 DstDir string `toml:"dstDir" json:"dstDir"` 45 FileNum int `toml:"fileNum" json:"fileNum"` 46 } 47 48 // SyncFileInfo records sync file progress 49 type SyncFileInfo struct { 50 Idx int `json:"idx"` 51 Location string `json:"loc"` 52 } 53 54 // Status records worker status of cvs job master 55 type Status struct { 56 *Config `json:"cfg"` 57 58 FileInfos map[int]*SyncFileInfo `json:"files"` 59 } 60 61 // WorkerInfo holds handler of worker 62 type WorkerInfo struct { 63 handle atomic.UnsafePointer // a handler to get information 64 needCreate atomic.Bool 65 } 66 67 // JobMaster defines cvs job master 68 type JobMaster struct { 69 sync.Mutex 70 71 framework.BaseJobMaster 72 jobStatus *Status 73 syncFilesInfo map[int]*WorkerInfo 74 counter int64 75 workerID frameModel.WorkerID 76 statusRateLimiter *rate.Limiter 77 78 launchedWorkers sync.Map 79 statusCode struct { 80 sync.RWMutex 81 code frameModel.WorkerState 82 } 83 ctx context.Context 84 clocker clock.Clock 85 } 86 87 var _ framework.JobMasterImpl = (*JobMaster)(nil) 88 89 // RegisterWorker is used to register cvs job master into global registry 90 func RegisterWorker() { 91 factory := registry.NewSimpleWorkerFactory(NewCVSJobMaster) 92 registry.GlobalWorkerRegistry().MustRegisterWorkerType(frameModel.CvsJobMaster, factory) 93 } 94 95 // NewCVSJobMaster creates a new cvs job master 96 func NewCVSJobMaster(ctx *dcontext.Context, workerID frameModel.WorkerID, masterID frameModel.MasterID, conf *Config) *JobMaster { 97 jm := &JobMaster{} 98 jm.workerID = workerID 99 jm.jobStatus = &Status{ 100 FileInfos: make(map[int]*SyncFileInfo), 101 } 102 jm.jobStatus.Config = conf 103 jm.syncFilesInfo = make(map[int]*WorkerInfo) 104 jm.statusRateLimiter = rate.NewLimiter(rate.Every(time.Second*2), 1) 105 jm.ctx = ctx.Context 106 jm.clocker = clock.New() 107 log.Info("new cvs jobmaster ", zap.Any("id :", jm.workerID)) 108 return jm 109 } 110 111 // InitImpl implements JobMasterImpl.InitImpl 112 func (jm *JobMaster) InitImpl(ctx context.Context) (err error) { 113 log.Info("initializing the cvs jobmaster ", zap.Any("id :", jm.workerID)) 114 jm.setState(frameModel.WorkerStateInit) 115 filesNum := jm.jobStatus.Config.FileNum 116 if filesNum == 0 { 117 return errors.New("no file found under the folder") 118 } 119 log.Info("cvs jobmaster list file success", zap.Any("id", jm.workerID), zap.Any("file number", filesNum)) 120 // todo: store the jobmaster information into the metastore 121 for idx := 0; idx < filesNum; idx++ { 122 jm.jobStatus.FileInfos[idx] = &SyncFileInfo{Idx: idx} 123 jm.syncFilesInfo[idx] = &WorkerInfo{ 124 needCreate: *atomic.NewBool(true), 125 handle: *atomic.NewUnsafePointer(unsafe.Pointer(nil)), 126 } 127 } 128 129 // Then persist the checkpoint for recovery 130 // This persistence has to succeed before we set this master to normal status. 131 statusBytes, err := json.Marshal(jm.jobStatus) 132 if err != nil { 133 return err 134 } 135 _, err = jm.MetaKVClient().Put(ctx, jm.workerID, string(statusBytes)) 136 if err != nil { 137 return err 138 } 139 jm.setState(frameModel.WorkerStateNormal) 140 return nil 141 } 142 143 // Tick implements JobMasterImpl.Tick 144 func (jm *JobMaster) Tick(ctx context.Context) error { 145 jm.counter = 0 146 if !jm.IsMasterReady() { 147 if jm.statusRateLimiter.Allow() { 148 log.Info("jobmaster is not ready", zap.Any("master id", jm.workerID)) 149 } 150 return nil 151 } 152 153 jm.Lock() 154 defer jm.Unlock() 155 if 0 == len(jm.jobStatus.FileInfos) { 156 jm.setState(frameModel.WorkerStateFinished) 157 log.Info("cvs job master finished") 158 status := jm.Status() 159 return jm.BaseJobMaster.Exit(ctx, framework.ExitReasonFinished, nil, status.ExtBytes) 160 } 161 for idx, workerInfo := range jm.syncFilesInfo { 162 // check if need to recreate worker 163 if workerInfo.needCreate.Load() { 164 workerID, err := jm.CreateWorker(frameModel.CvsTask, 165 getTaskConfig(jm.jobStatus, idx)) 166 if err != nil { 167 log.Warn("create worker failed, try next time", zap.Any("master id", jm.workerID), zap.Error(err)) 168 } else { 169 jm.launchedWorkers.Store(workerID, idx) 170 workerInfo.needCreate.Store(false) 171 } 172 continue 173 } 174 175 // still awaiting online 176 if workerInfo.handle.Load() == nil { 177 continue 178 } 179 // update job status 180 handle := *(*framework.WorkerHandle)(workerInfo.handle.Load()) 181 status := handle.Status() 182 switch status.State { 183 case frameModel.WorkerStateNormal, frameModel.WorkerStateFinished, frameModel.WorkerStateStopped: 184 taskStatus := &cvsTask.Status{} 185 err := json.Unmarshal(status.ExtBytes, taskStatus) 186 if err != nil { 187 return err 188 } 189 190 jm.jobStatus.FileInfos[idx].Location = taskStatus.CurrentLoc 191 jm.counter += taskStatus.Count 192 193 log.Debug("cvs job tmp num ", zap.Any("id", idx), zap.Any("status", string(status.ExtBytes))) 194 case frameModel.WorkerStateError: 195 log.Error("sync file failed ", zap.Any("idx", idx)) 196 default: 197 log.Info("worker status abnormal", zap.Any("status", status)) 198 } 199 } 200 if jm.statusRateLimiter.Allow() { 201 statsBytes, err := json.Marshal(jm.jobStatus) 202 if err != nil { 203 log.Warn("serialize job status failed, try next time", zap.Any("master id", jm.workerID), zap.Error(err)) 204 return err 205 } 206 _, err = jm.MetaKVClient().Put(ctx, jm.workerID, string(statsBytes)) 207 if err != nil { 208 log.Warn("update job status failed, try next time", zap.Any("master id", jm.workerID), zap.Error(err)) 209 } 210 log.Info("cvs job master status", zap.Any("id", jm.workerID), zap.Int64("counter", jm.counter), zap.Any("status", jm.getState())) 211 } 212 if jm.getState() == frameModel.WorkerStateStopped { 213 log.Info("cvs job master stopped") 214 status := jm.Status() 215 return jm.BaseJobMaster.Exit(ctx, framework.ExitReasonCanceled, nil, status.ExtBytes) 216 } 217 return nil 218 } 219 220 // OnMasterRecovered implements JobMasterImpl.OnMasterRecovered 221 func (jm *JobMaster) OnMasterRecovered(ctx context.Context) (err error) { 222 log.Info("recovering job master", zap.Any("id", jm.ID())) 223 // load self status 224 resp, err := jm.MetaKVClient().Get(ctx, jm.workerID) 225 if err != nil { 226 log.Warn("load status failed", zap.Any("master id", jm.ID), zap.Error(err)) 227 return err 228 } 229 if len(resp.Kvs) != 1 { 230 log.Error("jobmaster meta unexpected result", zap.Any("master id", jm.ID()), zap.Any("meta counts", len(resp.Kvs))) 231 } 232 statusBytes := resp.Kvs[0].Value 233 log.Info("jobmaster recover from meta", zap.Any("master id", jm.ID()), zap.String("status", string(statusBytes))) 234 err = json.Unmarshal(statusBytes, jm.jobStatus) 235 if err != nil { 236 return err 237 } 238 for id := range jm.jobStatus.FileInfos { 239 info := &WorkerInfo{} 240 info.needCreate.Store(true) 241 jm.syncFilesInfo[id] = info 242 } 243 return nil 244 } 245 246 // OnWorkerDispatched implements JobMasterImpl.OnWorkerDispatched 247 func (jm *JobMaster) OnWorkerDispatched(worker framework.WorkerHandle, err error) error { 248 if err == nil { 249 return nil 250 } 251 val, exist := jm.launchedWorkers.Load(worker.ID()) 252 log.Warn("Worker Dispatched Fail", zap.Any("master id", jm.ID()), zap.Any("worker id", err), zap.Error(err)) 253 if !exist { 254 log.Panic("failed worker not found", zap.Any("worker", worker.ID())) 255 } 256 jm.launchedWorkers.Delete(worker.ID()) 257 id := val.(int) 258 jm.Lock() 259 defer jm.Unlock() 260 jm.syncFilesInfo[id].needCreate.Store(true) 261 jm.syncFilesInfo[id].handle.Store(nil) 262 return nil 263 } 264 265 // OnWorkerOnline implements JobMasterImpl.OnWorkerOnline 266 func (jm *JobMaster) OnWorkerOnline(worker framework.WorkerHandle) error { 267 id, exist := jm.launchedWorkers.Load(worker.ID()) 268 if !exist { 269 log.Info("job master recovering and get new worker", zap.Any("id", worker.ID()), zap.Any("master id", jm.ID())) 270 if jm.IsMasterReady() { 271 log.Panic("job master has ready and a new worker has been created, brain split occurs!") 272 } 273 statusBytes := worker.Status().ExtBytes 274 status := cvsTask.Status{} 275 err := json.Unmarshal(statusBytes, &status) 276 if err != nil { 277 // bad json 278 return err 279 } 280 id = status.TaskConfig.Idx 281 } else { 282 log.Info("worker online ", zap.Any("id", worker.ID()), zap.Any("master id", jm.ID())) 283 } 284 jm.Lock() 285 defer jm.Unlock() 286 jm.syncFilesInfo[id.(int)].handle.Store(unsafe.Pointer(&worker)) 287 jm.syncFilesInfo[id.(int)].needCreate.Store(false) 288 jm.launchedWorkers.Store(worker.ID(), id.(int)) 289 return nil 290 } 291 292 func getTaskConfig(jobStatus *Status, id int) *cvsTask.Config { 293 return &cvsTask.Config{ 294 SrcHost: jobStatus.SrcHost, 295 DstHost: jobStatus.DstHost, 296 DstDir: jobStatus.DstDir, 297 StartLoc: jobStatus.FileInfos[id].Location, 298 Idx: id, 299 } 300 } 301 302 // OnWorkerOffline implements JobMasterImpl.OnWorkerOffline 303 // When offline, we should: 304 // 1. remove this file from map cache 305 // 2. update checkpoint, but note that this operation might fail. 306 func (jm *JobMaster) OnWorkerOffline(worker framework.WorkerHandle, reason error) error { 307 val, exist := jm.launchedWorkers.Load(worker.ID()) 308 log.Info("on worker offline ", zap.Any("worker", worker.ID())) 309 if !exist { 310 log.Panic("offline worker not found", zap.Any("worker", worker.ID())) 311 } 312 jm.launchedWorkers.Delete(worker.ID()) 313 id := val.(int) 314 jm.Lock() 315 defer jm.Unlock() 316 if errors.Is(reason, errors.ErrWorkerFinish) { 317 delete(jm.syncFilesInfo, id) 318 delete(jm.jobStatus.FileInfos, id) 319 log.Info("worker finished", zap.String("worker-id", worker.ID()), zap.Any("status", worker.Status()), zap.Error(reason)) 320 return nil 321 } 322 jm.syncFilesInfo[id].needCreate.Store(true) 323 jm.syncFilesInfo[id].handle.Store(nil) 324 return nil 325 } 326 327 // OnWorkerStatusUpdated implements JobMasterImpl.OnWorkerStatusUpdated 328 func (jm *JobMaster) OnWorkerStatusUpdated(worker framework.WorkerHandle, newStatus *frameModel.WorkerStatus) error { 329 return nil 330 } 331 332 // OnWorkerMessage implements JobMasterImpl.OnWorkerMessage 333 func (jm *JobMaster) OnWorkerMessage(worker framework.WorkerHandle, topic p2p.Topic, message p2p.MessageValue) error { 334 return nil 335 } 336 337 // CloseImpl is called when the master is being closed 338 func (jm *JobMaster) CloseImpl(ctx context.Context) {} 339 340 // StopImpl is called when the master is being canceled 341 func (jm *JobMaster) StopImpl(ctx context.Context) {} 342 343 // ID implements JobMasterImpl.ID 344 func (jm *JobMaster) ID() worker.RunnableID { 345 return jm.workerID 346 } 347 348 // OnMasterMessage implements JobMasterImpl.OnMasterMessage 349 func (jm *JobMaster) OnMasterMessage(ctx context.Context, topic p2p.Topic, message p2p.MessageValue) error { 350 return nil 351 } 352 353 // OnCancel implements JobMasterImpl.OnCancel 354 func (jm *JobMaster) OnCancel(ctx context.Context) error { 355 log.Info("cvs jobmaster: OnCancel") 356 return jm.cancelWorkers() 357 } 358 359 func (jm *JobMaster) cancelWorkers() error { 360 jm.setState(frameModel.WorkerStateStopped) 361 for _, worker := range jm.syncFilesInfo { 362 if worker.handle.Load() == nil { 363 continue 364 } 365 handle := *(*framework.WorkerHandle)(worker.handle.Load()) 366 workerID := handle.ID() 367 wTopic := frameModel.WorkerStatusChangeRequestTopic(jm.BaseJobMaster.ID(), handle.ID()) 368 wMessage := &frameModel.StatusChangeRequest{ 369 SendTime: jm.clocker.Mono(), 370 FromMasterID: jm.BaseJobMaster.ID(), 371 Epoch: jm.BaseJobMaster.CurrentEpoch(), 372 ExpectState: frameModel.WorkerStateStopped, 373 } 374 375 if handle := handle.Unwrap(); handle != nil { 376 ctx, cancel := context.WithTimeout(jm.ctx, time.Second*2) 377 if err := handle.SendMessage(ctx, wTopic, wMessage, false /*nonblocking*/); err != nil { 378 cancel() 379 return err 380 } 381 log.Info("sent message to worker", zap.String("topic", wTopic), zap.Any("message", wMessage)) 382 cancel() 383 } else { 384 log.Info("skip sending message to tombstone worker", zap.String("worker-id", workerID)) 385 } 386 } 387 return nil 388 } 389 390 // OnOpenAPIInitialized implements JobMasterImpl.OnOpenAPIInitialized. 391 func (jm *JobMaster) OnOpenAPIInitialized(apiGroup *gin.RouterGroup) {} 392 393 // Status implements JobMasterImpl.Status 394 func (jm *JobMaster) Status() frameModel.WorkerStatus { 395 status, err := json.Marshal(jm.jobStatus) 396 if err != nil { 397 log.Panic("get status failed", zap.String("id", jm.workerID), zap.Error(err)) 398 } 399 return frameModel.WorkerStatus{ 400 State: jm.getState(), 401 ExtBytes: status, 402 } 403 } 404 405 // IsJobMasterImpl implements JobMasterImpl.IsJobMasterImpl 406 func (jm *JobMaster) IsJobMasterImpl() { 407 panic("unreachable") 408 } 409 410 func (jm *JobMaster) setState(code frameModel.WorkerState) { 411 jm.statusCode.Lock() 412 defer jm.statusCode.Unlock() 413 jm.statusCode.code = code 414 } 415 416 func (jm *JobMaster) getState() frameModel.WorkerState { 417 jm.statusCode.RLock() 418 defer jm.statusCode.RUnlock() 419 return jm.statusCode.code 420 }