github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/executor/dm/unitholder.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 "sync" 19 "time" 20 21 dmconfig "github.com/pingcap/tiflow/dm/config" 22 "github.com/pingcap/tiflow/dm/dumpling" 23 "github.com/pingcap/tiflow/dm/loader" 24 "github.com/pingcap/tiflow/dm/pb" 25 "github.com/pingcap/tiflow/dm/pkg/binlog" 26 "github.com/pingcap/tiflow/dm/pkg/conn" 27 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 28 "github.com/pingcap/tiflow/dm/pkg/log" 29 "github.com/pingcap/tiflow/dm/syncer" 30 "github.com/pingcap/tiflow/dm/unit" 31 "github.com/pingcap/tiflow/engine/framework" 32 frameModel "github.com/pingcap/tiflow/engine/framework/model" 33 "github.com/pingcap/tiflow/engine/jobmaster/dm/metadata" 34 dmpkg "github.com/pingcap/tiflow/engine/pkg/dm" 35 "github.com/pingcap/tiflow/pkg/errors" 36 "go.uber.org/zap" 37 ) 38 39 // unitHolder hold a unit of DM 40 type unitHolder interface { 41 Init(ctx context.Context) error 42 Close(ctx context.Context) error 43 Pause(ctx context.Context) error 44 Resume(ctx context.Context) error 45 Stage() (metadata.TaskStage, *pb.ProcessResult) 46 Status(ctx context.Context) interface{} 47 // CheckAndUpdateStatus checks if the last update of source status is outdated, 48 // if so, it will call Status. 49 // this should be an async func. 50 CheckAndUpdateStatus() 51 Binlog(ctx context.Context, req *dmpkg.BinlogTaskRequest) (string, error) 52 BinlogSchema(ctx context.Context, req *dmpkg.BinlogSchemaTaskRequest) (string, error) 53 } 54 55 var ( 56 sourceStatusRefreshInterval = 30 * time.Second 57 sourceStatusCtxTimeOut = 20 * time.Second 58 ) 59 60 // unitHolderImpl wrap the dm-worker unit. 61 type unitHolderImpl struct { 62 tp framework.WorkerType 63 cfg *dmconfig.SubTaskConfig 64 unit unit.Unit 65 66 upstreamDB *conn.BaseDB 67 sourceStatus *binlog.SourceStatus 68 sourceStatusMu sync.RWMutex 69 sourceStatusCheckTime time.Time 70 71 logger log.Logger 72 // use to access process(init/close/pause/resume) 73 processMu sync.RWMutex 74 processWg sync.WaitGroup 75 // use to access field(ctx/result) 76 fieldMu sync.RWMutex 77 runCtx context.Context 78 runCancel context.CancelFunc 79 result *pb.ProcessResult // TODO: check if framework can persist result 80 81 // used to run background task 82 bgWg sync.WaitGroup 83 } 84 85 var _ unitHolder = &unitHolderImpl{} 86 87 // newUnitHolderImpl creates a UnitHolderImpl 88 func newUnitHolderImpl(workerType framework.WorkerType, cfg *dmconfig.SubTaskConfig) *unitHolderImpl { 89 return &unitHolderImpl{ 90 tp: workerType, 91 cfg: cfg, 92 } 93 } 94 95 // Init implement UnitHolder.Init 96 func (u *unitHolderImpl) Init(ctx context.Context) error { 97 u.processMu.Lock() 98 defer u.processMu.Unlock() 99 100 var err error 101 u.upstreamDB, err = conn.GetUpstreamDB(&u.cfg.From) 102 if err != nil { 103 return err 104 } 105 u.logger = log.Logger{Logger: u.cfg.FrameworkLogger}.WithFields( 106 zap.String("task", u.cfg.Name), zap.String("sourceID", u.cfg.SourceID), 107 ) 108 109 // worker may inject logger, metrics, etc. to config in InitImpl, so postpone construction 110 switch u.tp { 111 case frameModel.WorkerDMDump: 112 u.unit = dumpling.NewDumpling(u.cfg) 113 case frameModel.WorkerDMLoad: 114 sqlMode, err2 := conn.GetGlobalVariable(tcontext.NewContext(ctx, log.L()), u.upstreamDB, "sql_mode") 115 if err2 != nil { 116 u.logger.Error("get global sql_mode from upstream failed", 117 zap.String("db", u.cfg.From.Host), 118 zap.Int("port", u.cfg.From.Port), 119 zap.String("user", u.cfg.From.User), 120 zap.Error(err)) 121 return err2 122 } 123 u.cfg.LoaderConfig.SQLMode = sqlMode 124 u.unit = loader.NewLightning(u.cfg, nil, "dataflow-worker") 125 case frameModel.WorkerDMSync: 126 u.unit = syncer.NewSyncer(u.cfg, nil, nil) 127 } 128 129 if err = u.unit.Init(ctx); err != nil { 130 return err 131 } 132 133 runCtx, runCancel := context.WithCancel(context.Background()) 134 u.fieldMu.Lock() 135 u.runCtx, u.runCancel = runCtx, runCancel 136 u.fieldMu.Unlock() 137 138 resultCh := make(chan pb.ProcessResult, 1) 139 u.processWg.Add(1) 140 go func() { 141 defer u.processWg.Done() 142 u.unit.Process(runCtx, resultCh) 143 u.fetchAndHandleResult(resultCh) 144 }() 145 return nil 146 } 147 148 func (u *unitHolderImpl) Pause(ctx context.Context) error { 149 u.processMu.Lock() 150 defer u.processMu.Unlock() 151 152 stage, _ := u.Stage() 153 if stage != metadata.StageRunning && stage != metadata.StageError { 154 return errors.Errorf("failed to pause unit with stage %s", stage) 155 } 156 157 // cancel process 158 u.fieldMu.Lock() 159 u.runCancel() 160 u.fieldMu.Unlock() 161 u.bgWg.Wait() 162 u.processWg.Wait() 163 // TODO: refactor unit.Syncer 164 // unit needs to manage its own life cycle 165 u.unit.Pause() 166 return nil 167 } 168 169 func (u *unitHolderImpl) Resume(ctx context.Context) error { 170 u.processMu.Lock() 171 defer u.processMu.Unlock() 172 173 stage, _ := u.Stage() 174 if stage != metadata.StagePaused && stage != metadata.StageError { 175 return errors.Errorf("failed to resume unit with stage %s", stage) 176 } 177 178 runCtx, runCancel := context.WithCancel(context.Background()) 179 // run new process 180 u.fieldMu.Lock() 181 u.runCtx, u.runCancel = runCtx, runCancel 182 u.result = nil 183 u.fieldMu.Unlock() 184 185 resultCh := make(chan pb.ProcessResult, 1) 186 u.processWg.Add(1) 187 go func() { 188 defer u.processWg.Done() 189 u.unit.Resume(runCtx, resultCh) 190 u.fetchAndHandleResult(resultCh) 191 }() 192 return nil 193 } 194 195 // Close implement UnitHolder.Close 196 func (u *unitHolderImpl) Close(ctx context.Context) error { 197 u.processMu.Lock() 198 defer u.processMu.Unlock() 199 200 u.fieldMu.Lock() 201 // cancel process 202 if u.runCancel != nil { 203 u.runCancel() 204 } 205 u.fieldMu.Unlock() 206 207 u.bgWg.Wait() 208 u.processWg.Wait() 209 if u.unit != nil { 210 u.unit.Close() 211 } 212 return nil 213 } 214 215 // Stage implement UnitHolder.Stage 216 func (u *unitHolderImpl) Stage() (metadata.TaskStage, *pb.ProcessResult) { 217 u.fieldMu.RLock() 218 ctx := u.runCtx 219 result := u.result 220 u.fieldMu.RUnlock() 221 222 var canceled bool 223 select { 224 case <-ctx.Done(): 225 canceled = true 226 default: 227 } 228 229 switch { 230 case canceled && result == nil: 231 return metadata.StagePausing, nil 232 case canceled && result != nil: 233 return metadata.StagePaused, result 234 case !canceled && result == nil: 235 return metadata.StageRunning, nil 236 // !canceled && result != nil 237 case len(result.Errors) == 0: 238 return metadata.StageFinished, result 239 default: 240 return metadata.StageError, result 241 } 242 } 243 244 // Status implement UnitHolder.Status. Each invocation will try to query upstream 245 // once and calculate the status. 246 func (u *unitHolderImpl) Status(ctx context.Context) interface{} { 247 // nil sourceStatus is supported 248 return u.unit.Status(u.getSourceStatus()) 249 } 250 251 func (u *unitHolderImpl) updateSourceStatus(ctx context.Context) interface{} { 252 sourceStatus, err := binlog.GetSourceStatus( 253 tcontext.NewContext(ctx, u.logger), 254 u.upstreamDB, 255 u.cfg.Flavor, 256 ) 257 if err != nil { 258 u.logger.Warn("failed to get source status", zap.Error(err)) 259 } 260 u.setSourceStatus(sourceStatus) 261 return u.unit.Status(sourceStatus) 262 } 263 264 func (u *unitHolderImpl) getSourceStatus() *binlog.SourceStatus { 265 u.sourceStatusMu.RLock() 266 defer u.sourceStatusMu.RUnlock() 267 return u.sourceStatus 268 } 269 270 func (u *unitHolderImpl) setSourceStatus(in *binlog.SourceStatus) { 271 u.sourceStatusMu.Lock() 272 defer u.sourceStatusMu.Unlock() 273 u.sourceStatus = in 274 } 275 276 // CheckAndUpdateStatus implement UnitHolder.CheckAndUpdateStatus. 277 func (u *unitHolderImpl) CheckAndUpdateStatus() { 278 u.fieldMu.Lock() 279 defer u.fieldMu.Unlock() 280 if time.Since(u.sourceStatusCheckTime) > sourceStatusRefreshInterval { 281 u.sourceStatusCheckTime = time.Now() 282 u.bgWg.Add(1) 283 go func() { 284 defer u.bgWg.Done() 285 ctx, cancel := context.WithTimeout(context.Background(), sourceStatusCtxTimeOut) 286 u.updateSourceStatus(ctx) 287 cancel() 288 }() 289 } 290 } 291 292 // Binlog implements the binlog api for syncer unit. 293 func (u *unitHolderImpl) Binlog(ctx context.Context, req *dmpkg.BinlogTaskRequest) (string, error) { 294 syncUnit, ok := u.unit.(*syncer.Syncer) 295 if !ok { 296 return "", errors.Errorf("such operation is only available for syncer. current unit is %s", u.unit.Type()) 297 } 298 299 msg, err := syncUnit.HandleError(ctx, (*pb.HandleWorkerErrorRequest)(req)) 300 if err != nil { 301 return "", err 302 } 303 304 stage, _ := u.Stage() 305 if (stage == metadata.StagePaused || stage == metadata.StageError) && req.Op != pb.ErrorOp_List { 306 err = u.Resume(ctx) 307 } 308 return msg, err 309 } 310 311 // BinlogSchema implements the binlog schema api. 312 func (u *unitHolderImpl) BinlogSchema(ctx context.Context, req *dmpkg.BinlogSchemaTaskRequest) (string, error) { 313 syncUnit, ok := u.unit.(*syncer.Syncer) 314 if !ok { 315 return "", errors.Errorf("such operation is only available for syncer. current unit is %s", u.unit.Type()) 316 } 317 318 stage, _ := u.Stage() 319 if (stage != metadata.StagePaused && stage != metadata.StageError) && req.Op != pb.SchemaOp_ListMigrateTargets { 320 return "", errors.Errorf("current stage is %s but not paused, invalid", stage) 321 } 322 323 return syncUnit.OperateSchema(ctx, (*pb.OperateWorkerSchemaRequest)(req)) 324 } 325 326 func filterErrors(r *pb.ProcessResult) { 327 errs := make([]*pb.ProcessError, 0, 2) 328 for _, err := range r.Errors { 329 if !unit.IsCtxCanceledProcessErr(err) { 330 errs = append(errs, err) 331 } 332 } 333 r.Errors = errs 334 } 335 336 func (u *unitHolderImpl) fetchAndHandleResult(resultCh chan pb.ProcessResult) { 337 r := <-resultCh 338 filterErrors(&r) 339 if len(r.Errors) > 0 { 340 // TODO: refactor unit.Syncer 341 // unit needs to manage its own life cycle 342 u.unit.Pause() 343 } 344 u.fieldMu.Lock() 345 u.result = &r 346 u.fieldMu.Unlock() 347 }