github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/jobmaster/dm/api.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 dm 15 16 import ( 17 "context" 18 "fmt" 19 "sync" 20 "time" 21 22 frameModel "github.com/pingcap/tiflow/engine/framework/model" 23 "github.com/pingcap/tiflow/engine/jobmaster/dm/config" 24 "github.com/pingcap/tiflow/engine/jobmaster/dm/metadata" 25 "github.com/pingcap/tiflow/engine/jobmaster/dm/runtime" 26 dmpkg "github.com/pingcap/tiflow/engine/pkg/dm" 27 "github.com/pingcap/tiflow/pkg/errors" 28 ) 29 30 // TaskStatus represents status of a task 31 type TaskStatus struct { 32 ExpectedStage metadata.TaskStage `json:"expected_stage"` 33 WorkerID frameModel.WorkerID `json:"worker_id"` 34 ConfigOutdated bool `json:"config_outdated"` 35 Status *dmpkg.QueryStatusResponse `json:"status"` 36 Duration time.Duration `json:"duration"` 37 } 38 39 // JobStatus represents status of a job 40 type JobStatus struct { 41 JobID frameModel.MasterID `json:"job_id"` 42 // taskID -> Status 43 TaskStatus map[string]TaskStatus `json:"task_status"` 44 // FinishedUnitStatus records the finished unit status of a task. This field 45 // is not atomic with TaskStatus (current status). 46 FinishedUnitStatus map[string][]*metadata.FinishedTaskStatus `json:"finished_unit_status,omitempty"` 47 } 48 49 // ShardTable represents create table statements of a source table. 50 type ShardTable struct { 51 Current string 52 Next string 53 } 54 55 // DDLLock represents ddl lock of a target table. 56 type DDLLock struct { 57 // source table -> [current table, pending table(conflict table)] 58 ShardTables map[metadata.SourceTable]ShardTable 59 } 60 61 // ShowDDLLocksResponse represents response of show ddl locks. 62 type ShowDDLLocksResponse struct { 63 Locks map[metadata.TargetTable]DDLLock 64 } 65 66 // QueryJobStatus is the api of query job status. 67 func (jm *JobMaster) QueryJobStatus(ctx context.Context, tasks []string) (*JobStatus, error) { 68 jobState, err := jm.metadata.JobStore().Get(ctx) 69 if err != nil { 70 return nil, err 71 } 72 job := jobState.(*metadata.Job) 73 74 if len(tasks) == 0 { 75 for task := range job.Tasks { 76 tasks = append(tasks, task) 77 } 78 } 79 80 var expectedCfgModRevision uint64 81 for _, task := range job.Tasks { 82 expectedCfgModRevision = task.Cfg.ModRevision 83 break 84 } 85 86 var ( 87 workerStatusMap = jm.workerManager.WorkerStatus() 88 wg sync.WaitGroup 89 mu sync.Mutex 90 jobStatus = &JobStatus{ 91 JobID: jm.ID(), 92 TaskStatus: make(map[string]TaskStatus), 93 } 94 unitState *metadata.UnitState 95 existUnitState bool 96 ) 97 98 // need to get unit state here, so we calculate duration 99 state, err := jm.metadata.UnitStateStore().Get(ctx) 100 if err != nil && errors.Cause(err) != metadata.ErrStateNotFound { 101 return nil, err 102 } 103 unitState, existUnitState = state.(*metadata.UnitState) 104 for _, task := range tasks { 105 taskID := task 106 wg.Add(1) 107 go func() { 108 defer wg.Done() 109 110 var ( 111 queryStatusResp *dmpkg.QueryStatusResponse 112 workerID string 113 cfgModRevision uint64 114 expectedStage metadata.TaskStage 115 createdTime time.Time 116 duration time.Duration 117 ) 118 119 // task not exist 120 if t, ok := job.Tasks[taskID]; !ok { 121 queryStatusResp = &dmpkg.QueryStatusResponse{ErrorMsg: fmt.Sprintf("task %s for job not found", taskID)} 122 } else { 123 expectedStage = t.Stage 124 workerStatus, ok := workerStatusMap[taskID] 125 if !ok { 126 // worker unscheduled 127 queryStatusResp = &dmpkg.QueryStatusResponse{ErrorMsg: fmt.Sprintf("worker for task %s not found", taskID)} 128 } else if workerStatus.Stage != runtime.WorkerFinished { 129 workerID = workerStatus.ID 130 cfgModRevision = workerStatus.CfgModRevision 131 queryStatusResp = jm.QueryStatus(ctx, taskID) 132 } 133 } 134 135 if existUnitState { 136 if status, ok := unitState.CurrentUnitStatus[taskID]; ok { 137 createdTime = status.CreatedTime 138 } 139 } 140 if !createdTime.IsZero() { 141 duration = time.Since(createdTime) 142 } 143 144 mu.Lock() 145 jobStatus.TaskStatus[taskID] = TaskStatus{ 146 ExpectedStage: expectedStage, 147 WorkerID: workerID, 148 Status: queryStatusResp, 149 ConfigOutdated: cfgModRevision != expectedCfgModRevision, 150 Duration: duration, 151 } 152 mu.Unlock() 153 }() 154 } 155 wg.Wait() 156 157 // should be done after we get current task-status, since some unit status might be missing if 158 // current unit finish between we get finished unit status and get current task-status 159 state, err = jm.metadata.UnitStateStore().Get(ctx) 160 if err != nil && errors.Cause(err) != metadata.ErrStateNotFound { 161 return nil, err 162 } 163 unitState, existUnitState = state.(*metadata.UnitState) 164 if existUnitState { 165 jobStatus.FinishedUnitStatus = unitState.FinishedUnitStatus 166 } 167 168 return jobStatus, nil 169 } 170 171 // QueryStatus query status for a task 172 func (jm *JobMaster) QueryStatus(ctx context.Context, taskID string) *dmpkg.QueryStatusResponse { 173 req := &dmpkg.QueryStatusRequest{ 174 Task: taskID, 175 } 176 resp, err := jm.messageAgent.SendRequest(ctx, taskID, dmpkg.QueryStatus, req) 177 if err != nil { 178 return &dmpkg.QueryStatusResponse{ErrorMsg: err.Error()} 179 } 180 return resp.(*dmpkg.QueryStatusResponse) 181 } 182 183 // operateTask operate task. 184 func (jm *JobMaster) operateTask(ctx context.Context, op dmpkg.OperateType, cfg *config.JobCfg, tasks []string) error { 185 switch op { 186 case dmpkg.Resume, dmpkg.Pause, dmpkg.Update: 187 return jm.taskManager.OperateTask(ctx, op, cfg, tasks) 188 default: 189 return errors.Errorf("unsupported op type %d for operate task", op) 190 } 191 } 192 193 // GetJobCfg gets job config. 194 func (jm *JobMaster) GetJobCfg(ctx context.Context) (*config.JobCfg, error) { 195 return jm.metadata.JobStore().GetJobCfg(ctx) 196 } 197 198 // UpdateJobCfg updates job config. 199 func (jm *JobMaster) UpdateJobCfg(ctx context.Context, cfg *config.JobCfg) error { 200 if err := jm.preCheck(ctx, cfg); err != nil { 201 return err 202 } 203 if err := jm.operateTask(ctx, dmpkg.Update, cfg, nil); err != nil { 204 return err 205 } 206 // we don't know whether we can remove the old checkpoint, so we just create new checkpoint when update. 207 if err := jm.checkpointAgent.Create(ctx, cfg); err != nil { 208 return err 209 } 210 211 jm.workerManager.SetNextCheckTime(time.Now()) 212 return nil 213 } 214 215 // Binlog implements the api of binlog request. 216 func (jm *JobMaster) Binlog(ctx context.Context, req *dmpkg.BinlogRequest) (*dmpkg.BinlogResponse, error) { 217 if len(req.Sources) == 0 { 218 state, err := jm.metadata.JobStore().Get(ctx) 219 if err != nil { 220 return nil, err 221 } 222 job := state.(*metadata.Job) 223 for task := range job.Tasks { 224 req.Sources = append(req.Sources, task) 225 } 226 } 227 228 var ( 229 wg sync.WaitGroup 230 mu sync.Mutex 231 binlogResp = &dmpkg.BinlogResponse{ 232 Results: make(map[string]*dmpkg.CommonTaskResponse, len(req.Sources)), 233 } 234 ) 235 for _, task := range req.Sources { 236 taskID := task 237 wg.Add(1) 238 go func() { 239 defer wg.Done() 240 req := &dmpkg.BinlogTaskRequest{ 241 Op: req.Op, 242 BinlogPos: req.BinlogPos, 243 Sqls: req.Sqls, 244 } 245 resp := jm.BinlogTask(ctx, taskID, req) 246 mu.Lock() 247 binlogResp.Results[taskID] = resp 248 mu.Unlock() 249 }() 250 } 251 wg.Wait() 252 return binlogResp, nil 253 } 254 255 // BinlogTask implements the api of binlog task request. 256 func (jm *JobMaster) BinlogTask(ctx context.Context, taskID string, req *dmpkg.BinlogTaskRequest) *dmpkg.CommonTaskResponse { 257 // TODO: we may check the workerType via TaskManager/WorkerManger to reduce request connection. 258 resp, err := jm.messageAgent.SendRequest(ctx, taskID, dmpkg.BinlogTask, req) 259 if err != nil { 260 return &dmpkg.CommonTaskResponse{ErrorMsg: err.Error()} 261 } 262 return resp.(*dmpkg.CommonTaskResponse) 263 } 264 265 // BinlogSchema implements the api of binlog schema request. 266 func (jm *JobMaster) BinlogSchema(ctx context.Context, req *dmpkg.BinlogSchemaRequest) *dmpkg.BinlogSchemaResponse { 267 if len(req.Sources) == 0 { 268 return &dmpkg.BinlogSchemaResponse{ErrorMsg: "must specify at least one source"} 269 } 270 271 var ( 272 mu sync.Mutex 273 wg sync.WaitGroup 274 binlogSchemaResponse = &dmpkg.BinlogSchemaResponse{ 275 Results: make(map[string]*dmpkg.CommonTaskResponse, len(req.Sources)), 276 } 277 ) 278 for _, task := range req.Sources { 279 taskID := task 280 wg.Add(1) 281 go func() { 282 defer wg.Done() 283 req := &dmpkg.BinlogSchemaTaskRequest{ 284 Op: req.Op, 285 Source: taskID, 286 Database: req.Database, 287 Table: req.Table, 288 Schema: req.Schema, 289 Flush: req.Flush, 290 Sync: req.Sync, 291 FromSource: req.FromSource, 292 FromTarget: req.FromTarget, 293 } 294 resp := jm.BinlogSchemaTask(ctx, taskID, req) 295 mu.Lock() 296 binlogSchemaResponse.Results[taskID] = resp 297 mu.Unlock() 298 }() 299 } 300 wg.Wait() 301 return binlogSchemaResponse 302 } 303 304 // BinlogSchemaTask implements the api of binlog schema task request. 305 func (jm *JobMaster) BinlogSchemaTask(ctx context.Context, taskID string, req *dmpkg.BinlogSchemaTaskRequest) *dmpkg.CommonTaskResponse { 306 // TODO: we may check the workerType via TaskManager/WorkerManger to reduce request connection. 307 resp, err := jm.messageAgent.SendRequest(ctx, taskID, dmpkg.BinlogSchemaTask, req) 308 if err != nil { 309 return &dmpkg.CommonTaskResponse{ErrorMsg: err.Error()} 310 } 311 return resp.(*dmpkg.CommonTaskResponse) 312 } 313 314 // ShowDDLLocks implements the api of show ddl locks request. 315 func (jm *JobMaster) ShowDDLLocks(ctx context.Context) ShowDDLLocksResponse { 316 return jm.ddlCoordinator.ShowDDLLocks(ctx) 317 }