github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/master/bootstrap.go (about) 1 // Copyright 2020 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 master 15 16 import ( 17 "context" 18 "fmt" 19 "os" 20 "path/filepath" 21 "strings" 22 "sync" 23 "time" 24 25 "github.com/pingcap/errors" 26 "github.com/pingcap/tiflow/dm/config" 27 "github.com/pingcap/tiflow/dm/master/scheduler" 28 "github.com/pingcap/tiflow/dm/master/workerrpc" 29 "github.com/pingcap/tiflow/dm/pb" 30 "github.com/pingcap/tiflow/dm/pkg/conn" 31 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 32 "github.com/pingcap/tiflow/dm/pkg/log" 33 "github.com/pingcap/tiflow/dm/pkg/terror" 34 "github.com/pingcap/tiflow/dm/pkg/upgrade" 35 "github.com/pingcap/tiflow/dm/pkg/v1dbschema" 36 "github.com/pingcap/tiflow/dm/pkg/v1workermeta" 37 "go.uber.org/zap" 38 ) 39 40 var ( 41 // interval when waiting for the specified count of DM-worker instances become registered when importing from v1.0.x. 42 waitWorkerV1Interval = time.Second 43 // timeout when waiting for the specified count of DM-worker instances become registered when importing from v1.0.x. 44 waitWorkerV1Timeout = 5 * time.Minute 45 ) 46 47 // bootstrap bootstraps the cluster, now including: 48 // - upgrade the cluster from v1.0.x if needed. 49 // - upgrade the cluster from a previous v2.0.x version to the current version. 50 func (s *Server) bootstrap(ctx context.Context) error { 51 log.L().Info("start bootstrapping") 52 if s.cfg.V1SourcesPath != "" { 53 err := s.importFromV10x(ctx) 54 if err != nil { 55 return terror.ErrMasterFailToImportFromV10x.Delegate(err) 56 } 57 } else { 58 uctx := upgrade.Context{ 59 Context: ctx, 60 SubTaskConfigs: s.scheduler.GetSubTaskCfgs(), 61 } 62 err := upgrade.TryUpgrade(s.etcdClient, uctx) 63 if err != nil { 64 return err 65 } 66 } 67 log.L().Info("successful bootstrapped") 68 return nil 69 } 70 71 func (s *Server) bootstrapBeforeSchedulerStart(ctx context.Context) error { 72 log.L().Info("bootstrap before scheduler start") 73 // no need for v1.0.x 74 if s.cfg.V1SourcesPath != "" { 75 return nil 76 } 77 78 return upgrade.TryUpgradeBeforeSchedulerStart(ctx, s.etcdClient) 79 } 80 81 // importFromV10x tries to import/upgrade the cluster from v1.0.x. 82 func (s *Server) importFromV10x(ctx context.Context) error { 83 // 1. check whether need to upgrade based on the cluster version. 84 preVer, _, err := upgrade.GetVersion(s.etcdClient) 85 if err != nil { 86 return err 87 } else if !preVer.NotSet() { 88 return nil // v2.0.x cluster, no need to import. 89 } 90 91 logger := log.L().WithFields(zap.String("op", "import from v1.0.x")) 92 tctx := tcontext.NewContext(ctx, logger) 93 94 // 2. collect source config files. 95 logger.Info("collecting source config files", zap.String("path", s.cfg.V1SourcesPath)) 96 sourceCfgs, err := s.collectSourceConfigFilesV1Import(tctx) 97 if err != nil { 98 return err 99 } 100 101 // 3. wait for all DM-worker instances ready. 102 logger.Info("waiting for all DM-worker instances ready", zap.Int("count", len(sourceCfgs))) 103 err = s.waitWorkersReadyV1Import(tctx, sourceCfgs) 104 if err != nil { 105 return err 106 } 107 108 // 4. get subtasks config and stage from DM-worker instances. 109 logger.Info("getting subtask config and status from DM-worker") 110 subtaskCfgs, subtaskStages, err := s.getSubtaskCfgsStagesV1Import(tctx) 111 if err != nil { 112 return err 113 } 114 115 // 5. upgrade v1.0.x downstream metadata table and run v2.0 upgrading routines. 116 // some v2.0 upgrading routines are also altering schema, if we run them after adding sources, DM worker will 117 // meet error. 118 logger.Info("upgrading downstream metadata tables") 119 err = s.upgradeDBSchemaV1Import(tctx, subtaskCfgs) 120 if err != nil { 121 return err 122 } 123 uctx := upgrade.Context{ 124 Context: ctx, 125 SubTaskConfigs: subtaskCfgs, 126 } 127 err = upgrade.UntouchVersionUpgrade(s.etcdClient, uctx) 128 if err != nil { 129 return err 130 } 131 132 // 6. create sources. 133 logger.Info("add source config into cluster") 134 err = s.addSourcesV1Import(tctx, sourceCfgs) 135 if err != nil { 136 return err 137 } 138 139 // 7. create subtasks with the specified stage. 140 logger.Info("creating subtasks") 141 err = s.createSubtaskV1Import(tctx, subtaskCfgs, subtaskStages) 142 if err != nil { 143 return err 144 } 145 146 // 8. mark the upgrade operation as done. 147 logger.Info("marking upgrade from v1.0.x as done") 148 _, err = upgrade.PutVersion(s.etcdClient, upgrade.CurrentVersion) 149 if err != nil { 150 return err 151 } 152 153 // 9. clear v1.0.x data (source config files, DM-worker metadata), failed is not a problem. 154 logger.Info("clearing v1.0.x data") 155 s.clearOldDataV1Import(tctx) 156 157 // NOTE: add any other mechanisms to report the `done` of processing if needed later. 158 logger.Info("importing from v1.0.x has done") 159 return nil 160 } 161 162 // collectSourceConfigFilesV1Import tries to collect source config files for v1.0.x importing. 163 func (s *Server) collectSourceConfigFilesV1Import(tctx *tcontext.Context) (map[string]*config.SourceConfig, error) { 164 files, err := os.ReadDir(s.cfg.V1SourcesPath) 165 if err != nil { 166 return nil, err 167 } 168 169 cfgs := make(map[string]*config.SourceConfig) 170 for _, f := range files { 171 if f.IsDir() { 172 continue // ignore sub directories. 173 } 174 175 fp := filepath.Join(s.cfg.V1SourcesPath, f.Name()) 176 content, err := os.ReadFile(fp) 177 if err != nil { 178 return nil, err 179 } 180 181 cfgs2, err := parseAndAdjustSourceConfig(tctx.Ctx, []string{string(content)}) 182 if err != nil { 183 // abort importing if any invalid source config files exist. 184 return nil, err 185 } 186 187 cfgs[cfgs2[0].SourceID] = cfgs2[0] 188 tctx.Logger.Info("collected source config", zap.Stringer("config", cfgs2[0])) 189 } 190 191 return cfgs, nil 192 } 193 194 // waitWorkersReadyV1Import waits for DM-worker instances ready for v1.0.x importing. 195 // NOTE: in v1.0.x, `count of DM-worker instances` equals `count of source config files`. 196 func (s *Server) waitWorkersReadyV1Import(tctx *tcontext.Context, sourceCfgs map[string]*config.SourceConfig) error { 197 // now, we simply check count repeatedly. 198 count := len(sourceCfgs) 199 ctx2, cancel2 := context.WithTimeout(context.Background(), waitWorkerV1Timeout) 200 defer cancel2() 201 for { 202 select { 203 case <-tctx.Ctx.Done(): 204 return tctx.Ctx.Err() 205 case <-ctx2.Done(): 206 return errors.Errorf("wait for DM-worker instances timeout in %v", waitWorkerV1Timeout) 207 case <-time.After(waitWorkerV1Interval): 208 workers, err := s.scheduler.GetAllWorkers() 209 if err != nil { 210 return err 211 } 212 if len(workers) >= count { 213 tctx.Logger.Info("all DM-worker instances ready", zap.Int("ready count", len(workers))) 214 return nil 215 } 216 tctx.Logger.Info("waiting for DM-worker instances ready", zap.Int("ready count", len(workers))) 217 } 218 } 219 } 220 221 // getSubtaskCfgsStagesV1Import tries to get all subtask config and stage from DM-worker instances. 222 // returned: 223 // - configs: task-name -> source-ID -> subtask config. 224 // - stages: task-name -> source-ID -> subtask stage. 225 func (s *Server) getSubtaskCfgsStagesV1Import(tctx *tcontext.Context) ( 226 map[string]map[string]config.SubTaskConfig, map[string]map[string]pb.Stage, error, 227 ) { 228 workers, err := s.scheduler.GetAllWorkers() 229 if err != nil { 230 return nil, nil, err 231 } 232 233 req := workerrpc.Request{ 234 Type: workerrpc.CmdOperateV1Meta, 235 OperateV1Meta: &pb.OperateV1MetaRequest{Op: pb.V1MetaOp_GetV1Meta}, 236 } 237 respCh := make(chan *pb.OperateV1MetaResponse, len(workers)) 238 var wg sync.WaitGroup 239 for _, worker := range workers { 240 wg.Add(1) 241 go func(worker *scheduler.Worker) { 242 defer wg.Done() 243 resp, err := worker.SendRequest(tctx.Ctx, &req, s.cfg.RPCTimeout) 244 if err != nil { 245 respCh <- &pb.OperateV1MetaResponse{ 246 Result: false, 247 Msg: fmt.Sprintf("worker %s, %s", worker.BaseInfo().String(), err.Error()), 248 } 249 } else { 250 respCh <- resp.OperateV1Meta 251 } 252 }(worker) 253 } 254 wg.Wait() 255 256 subtaskCfgs := make(map[string]map[string]config.SubTaskConfig) 257 subtaskStages := make(map[string]map[string]pb.Stage) 258 errorMsgs := make([]string, 0) 259 for len(respCh) > 0 { 260 resp := <-respCh 261 if !resp.Result { 262 errorMsgs = append(errorMsgs, resp.Msg) 263 continue 264 } 265 266 for taskName, meta := range resp.Meta { 267 cfg, err := v1workermeta.SubTaskConfigFromV1TOML(meta.Task) 268 if err != nil { 269 tctx.Logger.Error("fail to get subtask config from v1 TOML", zap.ByteString("config", meta.Task)) 270 errorMsgs = append(errorMsgs, fmt.Sprintf("task %s, %s", meta.Name, err.Error())) 271 continue 272 } 273 274 if _, ok := subtaskCfgs[taskName]; !ok { 275 subtaskCfgs[taskName] = make(map[string]config.SubTaskConfig) 276 subtaskStages[taskName] = make(map[string]pb.Stage) 277 } 278 subtaskCfgs[taskName][cfg.SourceID] = cfg 279 subtaskStages[taskName][cfg.SourceID] = meta.Stage 280 tctx.Logger.Info("got subtask config and stage", zap.Stringer("config", &cfg), zap.Stringer("stage", meta.Stage)) 281 } 282 } 283 284 if len(errorMsgs) > 0 { 285 // if failed for any DM-worker instances, we abort the importing process now. 286 return nil, nil, errors.Errorf("fail to get subtask config and stage: %s", strings.Join(errorMsgs, ",")) 287 } 288 289 return subtaskCfgs, subtaskStages, nil 290 } 291 292 // addSourcesV1Import tries to add source config into the cluster for v1.0.x importing. 293 func (s *Server) addSourcesV1Import(tctx *tcontext.Context, cfgs map[string]*config.SourceConfig) error { 294 var ( 295 added []string 296 err error 297 ) 298 for _, cfg := range cfgs { 299 err = s.scheduler.AddSourceCfg(cfg) 300 if err != nil { 301 if terror.ErrSchedulerSourceCfgExist.Equal(err) { 302 err = nil // reset error 303 tctx.Logger.Warn("source already exists", zap.String("source", cfg.SourceID)) 304 } else { 305 break 306 } 307 } else { 308 added = append(added, cfg.SourceID) 309 } 310 } 311 312 if err != nil { 313 // try to remove source configs if any error occurred, but it's not a big problem if failed. 314 for _, sid := range added { 315 err2 := s.scheduler.RemoveSourceCfg(sid) 316 if err2 != nil { 317 tctx.Logger.Error("fail to remove source config", zap.String("source", sid), zap.Error(err2)) 318 } 319 } 320 } 321 return err 322 } 323 324 // upgradeDBSchemaV1Import tries to upgrade the metadata DB schema (and data) for v1.0.x importing. 325 func (s *Server) upgradeDBSchemaV1Import(tctx *tcontext.Context, cfgs map[string]map[string]config.SubTaskConfig) error { 326 for _, taskCfgs := range cfgs { 327 // all subtasks in one task must have the same downstream, so we only create one BaseDB instance. 328 // but different tasks may have different downstream. 329 var targetDB *conn.BaseDB 330 for _, cfg := range taskCfgs { 331 cfg2, err := cfg.DecryptedClone() // `cfg` should already be `Adjust`. 332 if err != nil { 333 return err 334 } 335 if targetDB == nil { 336 targetDB, err = conn.GetDownstreamDB(&cfg2.To) 337 if err != nil { 338 return err 339 } 340 } 341 err = v1dbschema.UpdateSchema(tctx, targetDB, cfg2) 342 if err != nil { 343 targetDB.Close() 344 return err 345 } 346 } 347 targetDB.Close() // close BaseDB for this task. 348 } 349 return nil 350 } 351 352 // createSubtaskV1Import tries to create subtasks with the specified stage. 353 // NOTE: we do not stopping previous subtasks if any later one failed (because some side effects may have taken), 354 // and let the user to check & fix the problem. 355 // TODO(csuzhangxc): merge subtask configs to support `get-task-config`. 356 func (s *Server) createSubtaskV1Import(tctx *tcontext.Context, 357 cfgs map[string]map[string]config.SubTaskConfig, stages map[string]map[string]pb.Stage, 358 ) error { 359 var err error 360 outerLoop: 361 for taskName, taskCfgs := range cfgs { 362 for sourceID, cfg := range taskCfgs { 363 var cfg2 *config.SubTaskConfig 364 cfg2, err = cfg.DecryptedClone() 365 if err != nil { 366 break outerLoop 367 } 368 stage := stages[taskName][sourceID] 369 switch stage { 370 case pb.Stage_Running, pb.Stage_Paused: 371 default: 372 tctx.Logger.Warn("skip to create subtask because only support to create subtasks with Running/Paused stage now", zap.Stringer("stage", stage)) 373 continue 374 } 375 err = s.scheduler.AddSubTasks(false, stage, *cfg2) 376 if err != nil { 377 if terror.ErrSchedulerSubTaskExist.Equal(err) { 378 err = nil // reset error 379 tctx.Logger.Warn("subtask already exists", zap.String("task", taskName), zap.String("source", sourceID)) 380 } else { 381 break outerLoop 382 } 383 } 384 } 385 } 386 return err 387 } 388 389 // clearOldDataV1Import tries to clear v1.0.x data after imported, now these data including: 390 // - source config files. 391 // - DM-worker metadata. 392 func (s *Server) clearOldDataV1Import(tctx *tcontext.Context) { 393 tctx.Logger.Info("removing source config files", zap.String("path", s.cfg.V1SourcesPath)) 394 err := os.RemoveAll(s.cfg.V1SourcesPath) 395 if err != nil { 396 tctx.Logger.Error("fail to remove source config files", zap.String("path", s.cfg.V1SourcesPath)) 397 } 398 399 workers, err := s.scheduler.GetAllWorkers() 400 if err != nil { 401 tctx.Logger.Error("fail to get DM-worker agents") 402 return 403 } 404 405 req := workerrpc.Request{ 406 Type: workerrpc.CmdOperateV1Meta, 407 OperateV1Meta: &pb.OperateV1MetaRequest{Op: pb.V1MetaOp_RemoveV1Meta}, 408 } 409 var wg sync.WaitGroup 410 for _, worker := range workers { 411 wg.Add(1) 412 go func(worker *scheduler.Worker) { 413 defer wg.Done() 414 tctx.Logger.Info("removing DM-worker metadata", zap.Stringer("worker", worker.BaseInfo())) 415 _, err2 := worker.SendRequest(tctx.Ctx, &req, s.cfg.RPCTimeout) 416 if err2 != nil { 417 tctx.Logger.Error("fail to remove metadata for DM-worker", zap.Stringer("worker", worker.BaseInfo())) 418 } 419 }(worker) 420 } 421 wg.Wait() 422 }