github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/engine/jobmaster/dm/checkpoint/agent.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 checkpoint 15 16 import ( 17 "bytes" 18 "context" 19 "database/sql" 20 "encoding/json" 21 "fmt" 22 "sync" 23 24 "github.com/coreos/go-semver/semver" 25 "github.com/pingcap/tidb-tools/pkg/dbutil" 26 "github.com/pingcap/tidb/pkg/executor" 27 "github.com/pingcap/tidb/pkg/meta/autoid" 28 "github.com/pingcap/tidb/pkg/parser/model" 29 "github.com/pingcap/tidb/pkg/util/filter" 30 "github.com/pingcap/tidb/pkg/util/mock" 31 regexprrouter "github.com/pingcap/tidb/pkg/util/regexpr-router" 32 router "github.com/pingcap/tidb/pkg/util/table-router" 33 dmconfig "github.com/pingcap/tiflow/dm/config" 34 "github.com/pingcap/tiflow/dm/pkg/conn" 35 "github.com/pingcap/tiflow/dm/pkg/cputil" 36 "github.com/pingcap/tiflow/engine/framework" 37 frameModel "github.com/pingcap/tiflow/engine/framework/model" 38 "github.com/pingcap/tiflow/engine/jobmaster/dm/bootstrap" 39 "github.com/pingcap/tiflow/engine/jobmaster/dm/config" 40 "github.com/pingcap/tiflow/engine/jobmaster/dm/metadata" 41 "github.com/pingcap/tiflow/pkg/errors" 42 "go.uber.org/zap" 43 "golang.org/x/sync/errgroup" 44 ) 45 46 const ( 47 loadCheckpointTable = `CREATE TABLE IF NOT EXISTS %s ( 48 task_name varchar(255) NOT NULL, 49 source_name varchar(255) NOT NULL, 50 status varchar(10) NOT NULL DEFAULT 'init' COMMENT 'init,running,finished', 51 PRIMARY KEY (task_name, source_name) 52 );` 53 syncCheckpointTable = `CREATE TABLE IF NOT EXISTS %s ( 54 id VARCHAR(32) NOT NULL, 55 cp_schema VARCHAR(128) NOT NULL, 56 cp_table VARCHAR(128) NOT NULL, 57 binlog_name VARCHAR(128), 58 binlog_pos INT UNSIGNED, 59 binlog_gtid TEXT, 60 exit_safe_binlog_name VARCHAR(128) DEFAULT '', 61 exit_safe_binlog_pos INT UNSIGNED DEFAULT 0, 62 exit_safe_binlog_gtid TEXT, 63 table_info JSON NOT NULL, 64 is_global BOOLEAN, 65 create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, 66 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 67 UNIQUE KEY uk_id_schema_table (id, cp_schema, cp_table) 68 )` 69 ) 70 71 // NewCheckpointAgent is a method to create a new checkpoint agent 72 var NewCheckpointAgent = NewAgentImpl 73 74 // Agent defeins a checkpoint agent interface 75 type Agent interface { 76 Create(ctx context.Context, cfg *config.JobCfg) error 77 Remove(ctx context.Context, cfg *config.JobCfg) error 78 IsFresh(ctx context.Context, workerType framework.WorkerType, task *metadata.Task) (bool, error) 79 Upgrade(ctx context.Context, preVer semver.Version) error 80 FetchAllDoTables(ctx context.Context, cfg *config.JobCfg) (map[metadata.TargetTable][]metadata.SourceTable, error) 81 // FetchTableStmt fetch create table statement from checkpoint. 82 FetchTableStmt(ctx context.Context, jobID string, cfg *config.JobCfg, sourceTable metadata.SourceTable) (string, error) 83 } 84 85 // AgentImpl implements Agent 86 type AgentImpl struct { 87 *bootstrap.DefaultUpgrader 88 89 jobID string 90 logger *zap.Logger 91 } 92 93 // NewAgentImpl creates a new AgentImpl instance 94 func NewAgentImpl(jobID string, pLogger *zap.Logger) Agent { 95 logger := pLogger.With(zap.String("component", "checkpoint_agent")) 96 c := &AgentImpl{ 97 DefaultUpgrader: bootstrap.NewDefaultUpgrader(logger), 98 jobID: jobID, 99 logger: logger, 100 } 101 c.DefaultUpgrader.Upgrader = c 102 return c 103 } 104 105 // Create implements Agent.Create 106 // Create will be called when the job is created/restarted/updated. 107 // We create checkpoint table in master rather than in workers, 108 // to avoid the annoying log of "table already exists", 109 // because one job only need to create one checkpoint table per unit. 110 // move these codes to tiflow later. 111 func (c *AgentImpl) Create(ctx context.Context, cfg *config.JobCfg) error { 112 c.logger.Info("create checkpoint") 113 db, err := conn.GetDownstreamDB(cfg.TargetDB) 114 if err != nil { 115 return errors.Trace(err) 116 } 117 defer db.Close() 118 119 if err := createMetaDatabase(ctx, cfg, db); err != nil { 120 return errors.Trace(err) 121 } 122 123 if dmconfig.HasLoad(cfg.TaskMode) { 124 if err := createLoadCheckpointTable(ctx, c.jobID, cfg, db); err != nil { 125 return errors.Trace(err) 126 } 127 } 128 if dmconfig.HasSync(cfg.TaskMode) { 129 if err := createSyncCheckpointTable(ctx, c.jobID, cfg, db); err != nil { 130 return errors.Trace(err) 131 } 132 } 133 return nil 134 } 135 136 // Remove implements Agent.Remove 137 func (c *AgentImpl) Remove(ctx context.Context, cfg *config.JobCfg) error { 138 c.logger.Info("remove checkpoint") 139 db, err := conn.GetDownstreamDB(cfg.TargetDB) 140 if err != nil { 141 return errors.Trace(err) 142 } 143 defer db.Close() 144 145 if err := dropLoadCheckpointTable(ctx, c.jobID, cfg, db); err != nil { 146 return errors.Trace(err) 147 } 148 return dropSyncCheckpointTable(ctx, c.jobID, cfg, db) 149 } 150 151 // IsFresh implements Agent.IsFresh 152 func (c *AgentImpl) IsFresh(ctx context.Context, workerType framework.WorkerType, task *metadata.Task) (bool, error) { 153 if workerType == frameModel.WorkerDMDump { 154 return true, nil 155 } 156 157 db, err := conn.GetDownstreamDB(task.Cfg.TargetDB) 158 if err != nil { 159 return false, err 160 } 161 defer db.Close() 162 163 if workerType == frameModel.WorkerDMLoad { 164 return isLoadFresh(ctx, c.jobID, task.Cfg, db) 165 } 166 return isSyncFresh(ctx, c.jobID, task.Cfg, db) 167 } 168 169 // UpgradeFuncs implement the Upgrader interface. 170 func (c *AgentImpl) UpgradeFuncs() []bootstrap.UpgradeFunc { 171 return nil 172 } 173 174 // FetchAllDoTables returns all need to do tables after filtered and routed (fetches from upstream MySQL). 175 func (c *AgentImpl) FetchAllDoTables(ctx context.Context, cfg *config.JobCfg) (map[metadata.TargetTable][]metadata.SourceTable, error) { 176 c.logger.Info("fetch all do tables from upstream") 177 178 var ( 179 mu sync.Mutex 180 result = make(map[metadata.TargetTable][]metadata.SourceTable, len(cfg.Upstreams)) 181 g, gCtx = errgroup.WithContext(ctx) 182 ) 183 184 for _, upstream := range cfg.Upstreams { 185 up := upstream 186 // filter/route in errgroup will cause data race. 187 baList, err := filter.New(up.CaseSensitive, cfg.BAList[up.BAListName]) 188 if err != nil { 189 return result, err 190 } 191 routeRules := make([]*router.TableRule, 0, len(up.RouteRules)) 192 for _, ruleName := range up.RouteRules { 193 routeRules = append(routeRules, cfg.Routes[ruleName]) 194 } 195 router, err := regexprrouter.NewRegExprRouter(up.CaseSensitive, routeRules) 196 if err != nil { 197 return result, err 198 } 199 200 g.Go(func() error { 201 db, err := conn.DefaultDBProvider.Apply(conn.UpstreamDBConfig(up.DBCfg)) 202 if err != nil { 203 return errors.Trace(err) 204 } 205 defer db.Close() 206 207 // fetch all do tables 208 sourceTables, err := conn.FetchAllDoTables(gCtx, db, baList) 209 if err != nil { 210 return errors.Trace(err) 211 } 212 213 for schema, tables := range sourceTables { 214 for _, table := range tables { 215 targetSchema, targetTable, err := router.Route(schema, table) 216 if err != nil { 217 return errors.Trace(err) 218 } 219 220 target := metadata.TargetTable{Schema: targetSchema, Table: targetTable} 221 mu.Lock() 222 result[target] = append(result[target], metadata.SourceTable{Source: up.SourceID, Schema: schema, Table: table}) 223 mu.Unlock() 224 } 225 } 226 return nil 227 }) 228 229 } 230 err := g.Wait() 231 return result, err 232 } 233 234 // FetchTableStmt fetch create table statement from checkpoint. 235 // TODO(https://github.com/pingcap/tiflow/issues/5334): save create table statement to checkpoint instead of table info. 236 func (c *AgentImpl) FetchTableStmt(ctx context.Context, jobID string, cfg *config.JobCfg, sourceTable metadata.SourceTable) (string, error) { 237 c.logger.Info("fetch table info from checkpoint") 238 db, err := conn.DefaultDBProvider.Apply(conn.DownstreamDBConfig(cfg.TargetDB)) 239 if err != nil { 240 return "", err 241 } 242 defer db.Close() 243 244 query := `SELECT table_info FROM ` + syncTableName(jobID, cfg) + ` WHERE id = ? AND cp_schema = ? AND cp_table = ?` 245 row := db.DB.QueryRowContext(ctx, query, sourceTable.Source, sourceTable.Schema, sourceTable.Table) 246 if row.Err() != nil { 247 return "", row.Err() 248 } 249 var tiBytes []byte 250 if err := row.Scan(&tiBytes); err != nil { 251 if err == sql.ErrNoRows { 252 return "", errors.Errorf("table info not found %v", sourceTable) 253 } 254 return "", err 255 } 256 var ti *model.TableInfo 257 if bytes.Equal(tiBytes, []byte("null")) { 258 return "", errors.Errorf("table info not found %v", sourceTable) 259 } 260 if err := json.Unmarshal(tiBytes, &ti); err != nil { 261 return "", errors.Trace(err) 262 } 263 264 result := bytes.NewBuffer(make([]byte, 0, 512)) 265 err = executor.ConstructResultOfShowCreateTable(mock.NewContext(), ti, autoid.Allocators{}, result) 266 if err != nil { 267 return "", errors.Trace(err) 268 } 269 return conn.CreateTableSQLToOneRow(result.String()), nil 270 } 271 272 func createMetaDatabase(ctx context.Context, cfg *config.JobCfg, db *conn.BaseDB) error { 273 query := fmt.Sprintf("CREATE DATABASE IF NOT EXISTS %s", dbutil.ColumnName(cfg.MetaSchema)) 274 _, err := db.DB.ExecContext(ctx, query) 275 return errors.Trace(err) 276 } 277 278 func createLoadCheckpointTable(ctx context.Context, jobID string, cfg *config.JobCfg, db *conn.BaseDB) error { 279 _, err := db.DB.ExecContext(ctx, fmt.Sprintf(loadCheckpointTable, loadTableName(jobID, cfg))) 280 return errors.Trace(err) 281 } 282 283 func createSyncCheckpointTable(ctx context.Context, jobID string, cfg *config.JobCfg, db *conn.BaseDB) error { 284 _, err := db.DB.ExecContext(ctx, fmt.Sprintf(syncCheckpointTable, syncTableName(jobID, cfg))) 285 return errors.Trace(err) 286 } 287 288 func dropLoadCheckpointTable(ctx context.Context, jobID string, cfg *config.JobCfg, db *conn.BaseDB) error { 289 dropTable := "DROP TABLE IF EXISTS %s" 290 _, err := db.DB.ExecContext(ctx, fmt.Sprintf(dropTable, loadTableName(jobID, cfg))) 291 return errors.Trace(err) 292 } 293 294 func dropSyncCheckpointTable(ctx context.Context, jobID string, cfg *config.JobCfg, db *conn.BaseDB) error { 295 dropTable := "DROP TABLE IF EXISTS %s" 296 if _, err := db.DB.ExecContext(ctx, fmt.Sprintf(dropTable, syncTableName(jobID, cfg))); err != nil { 297 return errors.Trace(err) 298 } 299 // The following two would be better removed in the worker when destroy. 300 if _, err := db.DB.ExecContext(ctx, fmt.Sprintf(dropTable, shardMetaName(jobID, cfg))); err != nil { 301 return errors.Trace(err) 302 } 303 if _, err := db.DB.ExecContext(ctx, fmt.Sprintf(dropTable, onlineDDLName(jobID, cfg))); err != nil { 304 return errors.Trace(err) 305 } 306 return nil 307 } 308 309 func loadTableName(jobID string, cfg *config.JobCfg) string { 310 return dbutil.TableName(cfg.MetaSchema, cputil.LightningCheckpoint(jobID)) 311 } 312 313 func syncTableName(jobID string, cfg *config.JobCfg) string { 314 return dbutil.TableName(cfg.MetaSchema, cputil.SyncerCheckpoint(jobID)) 315 } 316 317 func shardMetaName(jobID string, cfg *config.JobCfg) string { 318 return dbutil.TableName(cfg.MetaSchema, cputil.SyncerShardMeta(jobID)) 319 } 320 321 func onlineDDLName(jobID string, cfg *config.JobCfg) string { 322 return dbutil.TableName(cfg.MetaSchema, cputil.SyncerOnlineDDL(jobID)) 323 } 324 325 func isLoadFresh(ctx context.Context, jobID string, taskCfg *config.TaskCfg, db *conn.BaseDB) (bool, error) { 326 // nolint:gosec 327 query := fmt.Sprintf("SELECT status FROM %s WHERE `task_name` = ? AND `source_name` = ?", loadTableName(jobID, taskCfg.ToJobCfg())) 328 var status string 329 err := db.DB.QueryRowContext(ctx, query, jobID, taskCfg.Upstreams[0].SourceID).Scan(&status) 330 switch { 331 case err == nil: 332 return status == "init", nil 333 case err == sql.ErrNoRows: 334 return true, nil 335 default: 336 return false, err 337 } 338 } 339 340 func isSyncFresh(ctx context.Context, jobID string, taskCfg *config.TaskCfg, db *conn.BaseDB) (bool, error) { 341 // nolint:gosec 342 query := fmt.Sprintf("SELECT 1 FROM %s WHERE `id` = ? AND `is_global` = true", syncTableName(jobID, taskCfg.ToJobCfg())) 343 var status string 344 err := db.DB.QueryRowContext(ctx, query, taskCfg.Upstreams[0].SourceID).Scan(&status) 345 switch { 346 case err == nil: 347 return false, nil 348 case err == sql.ErrNoRows: 349 return true, nil 350 default: 351 return false, err 352 } 353 }