github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/checkpoint.go (about) 1 // Copyright 2019 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 syncer 15 16 import ( 17 "bytes" 18 "context" 19 "database/sql" 20 "encoding/json" 21 "fmt" 22 "io" 23 "strings" 24 "sync" 25 "time" 26 27 "github.com/go-mysql-org/go-mysql/mysql" 28 "github.com/pingcap/errors" 29 "github.com/pingcap/failpoint" 30 "github.com/pingcap/tidb/pkg/parser/model" 31 tmysql "github.com/pingcap/tidb/pkg/parser/mysql" 32 "github.com/pingcap/tidb/pkg/util/dbutil" 33 "github.com/pingcap/tidb/pkg/util/filter" 34 "github.com/pingcap/tiflow/dm/config" 35 "github.com/pingcap/tiflow/dm/config/dbconfig" 36 "github.com/pingcap/tiflow/dm/pkg/binlog" 37 "github.com/pingcap/tiflow/dm/pkg/conn" 38 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 39 "github.com/pingcap/tiflow/dm/pkg/cputil" 40 "github.com/pingcap/tiflow/dm/pkg/dumpling" 41 fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" 42 "github.com/pingcap/tiflow/dm/pkg/gtid" 43 "github.com/pingcap/tiflow/dm/pkg/log" 44 "github.com/pingcap/tiflow/dm/pkg/schema" 45 "github.com/pingcap/tiflow/dm/pkg/storage" 46 "github.com/pingcap/tiflow/dm/pkg/terror" 47 "github.com/pingcap/tiflow/dm/syncer/dbconn" 48 "github.com/pingcap/tiflow/dm/syncer/metrics" 49 "github.com/uber-go/atomic" 50 "go.uber.org/zap" 51 ) 52 53 /* 54 variants about checkpoint: 55 1. update global checkpoint for DDL/XID event from any stream (global and sharding streaming) 56 2. update table checkpoint for DDL/DML event from any stream (global and sharding streaming) 57 3. position of global/table checkpoint increases monotonically 58 4. global checkpoint <= min checkpoint of all unsolved sharding tables 59 5. max checkpoint of all tables >= global checkpoint 60 */ 61 62 var ( 63 globalCpSchema = "" // global checkpoint's cp_schema 64 globalCpTable = "" // global checkpoint's cp_table 65 maxCheckPointTimeout = "1m" 66 batchFlushPoints = 100 67 ) 68 69 type tablePoint struct { 70 location binlog.Location 71 ti *model.TableInfo 72 } 73 74 func (t *tablePoint) writeString(buf io.Writer) { 75 if t == nil { 76 return 77 } 78 fmt.Fprintf(buf, "location(%s)", t.location) 79 if t.ti != nil { 80 fmt.Fprintf(buf, ", tableInfo(Name:%s, ColNum: %d, IdxNum: %d, PKIsHandle: %t)", t.ti.Name, len(t.ti.Columns), len(t.ti.Indices), t.ti.PKIsHandle) 81 } 82 } 83 84 func (t *tablePoint) String() string { 85 if t == nil { 86 return "" 87 } 88 var buf strings.Builder 89 t.writeString(&buf) 90 return buf.String() 91 } 92 93 type binlogPoint struct { 94 sync.RWMutex 95 96 savedPoint tablePoint 97 flushedPoint tablePoint // point which flushed persistently 98 enableGTID bool 99 } 100 101 func newBinlogPoint(location, flushedLocation binlog.Location, ti, flushedTI *model.TableInfo, enableGTID bool) *binlogPoint { 102 return &binlogPoint{ 103 savedPoint: tablePoint{ 104 location: location, 105 ti: ti, 106 }, 107 flushedPoint: tablePoint{ 108 location: flushedLocation, 109 ti: flushedTI, 110 }, 111 enableGTID: enableGTID, 112 } 113 } 114 115 func (b *binlogPoint) save(location binlog.Location, ti *model.TableInfo) error { 116 b.Lock() 117 defer b.Unlock() 118 119 if binlog.CompareLocation(location, b.savedPoint.location, b.enableGTID) < 0 { 120 // support to save equal location, but not older location 121 return terror.ErrCheckpointSaveInvalidPos.Generate(location, b.savedPoint.location) 122 } 123 124 b.savedPoint.location = location 125 b.savedPoint.ti = ti 126 return nil 127 } 128 129 func (b *binlogPoint) flush() { 130 b.flushBy(b.savedPoint) 131 } 132 133 func (b *binlogPoint) flushBy(tp tablePoint) { 134 b.Lock() 135 defer b.Unlock() 136 b.flushedPoint = tp 137 } 138 139 func (b *binlogPoint) rollback() { 140 b.Lock() 141 defer b.Unlock() 142 143 // set suffix to 0 when we meet error 144 b.flushedPoint.location.ResetSuffix() 145 b.savedPoint.location = b.flushedPoint.location 146 if b.savedPoint.ti == nil { 147 // TODO: if we forget to save table info for table checkpoint, this is also nil! 148 // And table checkpoint rollback to flushed point may also be nil! 149 return // for global checkpoint, no need to rollback the schema. 150 } 151 152 // NOTE: no `Equal` function for `model.TableInfo` exists now, so we compare `pointer` directly, 153 // and after a new DDL applied to the schema, the returned pointer of `model.TableInfo` changed now. 154 // there may be three versions of schema: 155 // - the one tracked in the schema tracker (TiDB-with-unistore). 156 // - the one in the checkpoint but not flushed. 157 // - the one in the checkpoint and flushed. 158 // schema tracker will be closed after task is paused, and it will load all schemas from checkpoint when task resumes. 159 // if the later two are not equal, then we rollback them: 160 // - set the one in the checkpoint but not flushed to the one flushed. 161 if b.savedPoint.ti != b.flushedPoint.ti { 162 b.savedPoint.ti = b.flushedPoint.ti 163 } 164 } 165 166 func (b *binlogPoint) outOfDate() bool { 167 return b.outOfDateBy(b.savedPoint.location) 168 } 169 170 func (b *binlogPoint) outOfDateBy(pos binlog.Location) bool { 171 b.RLock() 172 defer b.RUnlock() 173 174 return binlog.CompareLocation(pos, b.flushedPoint.location, b.enableGTID) > 0 175 } 176 177 // MySQLLocation returns point as binlog.Location. 178 func (b *binlogPoint) MySQLLocation() binlog.Location { 179 b.RLock() 180 defer b.RUnlock() 181 return b.savedPoint.location 182 } 183 184 // FlushedMySQLLocation returns flushed point as binlog.Location. 185 func (b *binlogPoint) FlushedMySQLLocation() binlog.Location { 186 b.RLock() 187 defer b.RUnlock() 188 return b.flushedPoint.location 189 } 190 191 // TableInfo returns the table schema associated at the current binlog position. 192 func (b *binlogPoint) TableInfo() *model.TableInfo { 193 b.RLock() 194 defer b.RUnlock() 195 return b.savedPoint.ti 196 } 197 198 func (b *binlogPoint) String() string { 199 b.RLock() 200 defer b.RUnlock() 201 202 var buf strings.Builder 203 b.savedPoint.writeString(&buf) 204 buf.WriteString(" (flushed ") 205 b.flushedPoint.writeString(&buf) 206 buf.WriteString(")") 207 208 return buf.String() 209 } 210 211 // SnapshotInfo contains: 212 // - checkpoint snapshot id, it's for retrieving checkpoint snapshot in flush phase 213 // - global checkpoint position, it's for updating current active relay log after checkpoint flush. 214 type SnapshotInfo struct { 215 // the snapshot id 216 id int 217 // global checkpoint position. 218 globalPos binlog.Location 219 } 220 221 // CheckPoint represents checkpoints status for syncer 222 // including global binlog's checkpoint and every table's checkpoint 223 // when save checkpoint, we must differ saving in memory from saving (flushing) to DB (or file) permanently 224 // for sharding merging, we must save checkpoint in memory to support skip when re-syncing for the special streamer 225 // but before all DDLs for a sharding group to be synced and executed, we should not save checkpoint permanently 226 // because, when restarting to continue the sync, all sharding DDLs must try-sync again. 227 type CheckPoint interface { 228 // Init initializes the CheckPoint 229 Init(tctx *tcontext.Context) error 230 231 // Close closes the CheckPoint 232 Close() 233 234 // ResetConn resets database connections owned by the Checkpoint 235 ResetConn(tctx *tcontext.Context) error 236 237 // Clear clears all checkpoints 238 Clear(tctx *tcontext.Context) error 239 240 // Load loads all checkpoints saved by CheckPoint 241 Load(tctx *tcontext.Context) error 242 243 // LoadMeta loads checkpoints from meta config item or file 244 LoadMeta(ctx context.Context) error 245 246 // SaveTablePoint saves checkpoint for specified table in memory 247 SaveTablePoint(table *filter.Table, point binlog.Location, ti *model.TableInfo) 248 249 // DeleteTablePoint deletes checkpoint for specified table in memory and storage 250 DeleteTablePoint(tctx *tcontext.Context, table *filter.Table) error 251 252 // DeleteAllTablePoint deletes all checkpoints for table in memory and storage 253 DeleteAllTablePoint(tctx *tcontext.Context) error 254 255 // DeleteSchemaPoint deletes checkpoint for specified schema 256 DeleteSchemaPoint(tctx *tcontext.Context, sourceSchema string) error 257 258 // IsOlderThanTablePoint checks whether job's checkpoint is older than previous saved checkpoint 259 IsOlderThanTablePoint(table *filter.Table, point binlog.Location) bool 260 261 // SaveGlobalPoint saves the global binlog stream's checkpoint 262 // corresponding to Meta.Save 263 SaveGlobalPoint(point binlog.Location) 264 265 // SaveGlobalPointForcibly saves the global binlog stream's checkpoint forcibly. 266 SaveGlobalPointForcibly(location binlog.Location) 267 268 // Snapshot make a snapshot of current checkpoint. If returns nil, it means nothing has changed since last call. 269 Snapshot(isSyncFlush bool) *SnapshotInfo 270 271 // DiscardPendingSnapshots discards all pending snapshots. It's used when we create a snapshot but are unable to 272 // call FlushPointsExcept() to flush the snapshot due to some error. 273 DiscardPendingSnapshots() 274 275 // FlushPointsExcept flushes the global checkpoint and tables' 276 // checkpoints except exceptTables, it also flushes SQLs with Args providing 277 // by extraSQLs and extraArgs. Currently extraSQLs contain shard meta only. 278 // @exceptTables: [[schema, table]... ] 279 // corresponding to Meta.Flush 280 FlushPointsExcept(tctx *tcontext.Context, snapshotID int, exceptTables []*filter.Table, extraSQLs []string, extraArgs [][]interface{}) error 281 282 // FlushPointsWithTableInfos flushed the table points with given table infos 283 FlushPointsWithTableInfos(tctx *tcontext.Context, tables []*filter.Table, tis []*model.TableInfo) error 284 285 // FlushSafeModeExitPoint flushed the global checkpoint's with given table info 286 FlushSafeModeExitPoint(tctx *tcontext.Context) error 287 288 // GlobalPoint returns the global binlog stream's checkpoint 289 // corresponding to Meta.Pos and Meta.GTID 290 GlobalPoint() binlog.Location 291 292 // GlobalPointSaveTime return the global point saved time, used for test only 293 GlobalPointSaveTime() time.Time 294 295 // SaveSafeModeExitPoint saves the pointer to location which indicates safe mode exit 296 // this location is used when dump unit can't assure consistency 297 SaveSafeModeExitPoint(point *binlog.Location) 298 299 // SafeModeExitPoint returns the location where safe mode could safely turn off after 300 SafeModeExitPoint() *binlog.Location 301 302 // TablePoint returns all table's stream checkpoint 303 TablePoint() map[string]map[string]binlog.Location 304 305 // GetTableInfo returns the saved table info from table checkpoint for the given table, return nil when not found 306 GetTableInfo(schema string, table string) *model.TableInfo 307 308 // FlushedGlobalPoint returns the flushed global binlog stream's checkpoint 309 // corresponding to to Meta.Pos and gtid 310 FlushedGlobalPoint() binlog.Location 311 312 // LastFlushOutdated checks the start time of a flush (when call Snapshot) and finish time of a flush, if both of 313 // the two times are outdated, LastFlushOutdated returns true. 314 LastFlushOutdated() bool 315 316 // Rollback rolls global checkpoint and all table checkpoints back to flushed checkpoints 317 Rollback() 318 319 // String return text of global position 320 String() string 321 322 // LoadIntoSchemaTracker loads table infos of all points into schema tracker. 323 LoadIntoSchemaTracker(ctx context.Context, schemaTracker *schema.Tracker) error 324 325 // CheckAndUpdate check the checkpoint data consistency and try to fix them if possible 326 CheckAndUpdate(ctx context.Context, schemas map[string]string, tables map[string]map[string]string) error 327 } 328 329 // remoteCheckpointSnapshot contains info needed to flush checkpoint to downstream by FlushPointsExcept method. 330 type remoteCheckpointSnapshot struct { 331 id int 332 globalPoint *tablePoint 333 globalPointSaveTime time.Time 334 points map[string]map[string]tablePoint 335 } 336 337 // RemoteCheckPoint implements CheckPoint 338 // which using target database to store info 339 // NOTE: now we sync from relay log, so not add GTID support yet 340 // it's not thread-safe. 341 type RemoteCheckPoint struct { 342 sync.RWMutex 343 344 cfg *config.SubTaskConfig 345 metricProxies *metrics.Proxies 346 347 db *conn.BaseDB 348 dbConn *dbconn.DBConn 349 tableName string // qualified table name: schema is set through task config, table is task name 350 id string // checkpoint ID, now it is `source-id` 351 352 // source-schema -> source-table -> checkpoint 353 // used to filter the synced binlog when re-syncing for sharding group 354 points map[string]map[string]*binlogPoint 355 356 // global binlog checkpoint 357 // after restarted, we can continue to re-sync from this point 358 // if there are sharding groups waiting for DDL syncing or in DMLs re-syncing 359 // this global checkpoint is min(next-binlog-pos, min(all-syncing-sharding-group-first-pos)) 360 // else 361 // this global checkpoint is next-binlog-pos 362 globalPoint *binlogPoint 363 globalPointSaveTime time.Time 364 lastSnapshotCreationTime time.Time 365 366 // safeModeExitPoint is set in RemoteCheckPoint.Load (from downstream DB) and LoadMeta (from metadata file). 367 // it is unset (set nil) in RemoteCheckPoint.Clear, and when syncer's stream pass its location. 368 // it is flushed along with globalPoint which called by Syncer.flushCheckPoints. 369 // this variable is mainly used to decide status of safe mode, so it is access when 370 // - init safe mode 371 // - checking in sync and if passed, unset it 372 safeModeExitPoint *binlog.Location 373 needFlushSafeModeExitPoint atomic.Bool 374 375 logCtx *tcontext.Context 376 377 snapshots []*remoteCheckpointSnapshot 378 snapshotSeq int 379 } 380 381 // NewRemoteCheckPoint creates a new RemoteCheckPoint. 382 func NewRemoteCheckPoint( 383 tctx *tcontext.Context, 384 cfg *config.SubTaskConfig, 385 metricProxies *metrics.Proxies, 386 id string, 387 ) CheckPoint { 388 cp := &RemoteCheckPoint{ 389 cfg: cfg, 390 metricProxies: metricProxies, 391 tableName: dbutil.TableName(cfg.MetaSchema, cputil.SyncerCheckpoint(cfg.Name)), 392 id: id, 393 points: make(map[string]map[string]*binlogPoint), 394 globalPoint: newBinlogPoint(binlog.MustZeroLocation(cfg.Flavor), binlog.MustZeroLocation(cfg.Flavor), nil, nil, cfg.EnableGTID), 395 logCtx: tcontext.Background().WithLogger(tctx.L().WithFields(zap.String("component", "remote checkpoint"))), 396 snapshots: make([]*remoteCheckpointSnapshot, 0), 397 snapshotSeq: 0, 398 } 399 400 return cp 401 } 402 403 // Snapshot make a snapshot of checkpoint and return the snapshot info. 404 func (cp *RemoteCheckPoint) Snapshot(isSyncFlush bool) *SnapshotInfo { 405 cp.RLock() 406 defer cp.RUnlock() 407 408 // make snapshot is visit in single thread, so depend on rlock should be enough 409 cp.snapshotSeq++ 410 id := cp.snapshotSeq 411 cp.lastSnapshotCreationTime = time.Now() 412 413 tableCheckPoints := make(map[string]map[string]tablePoint, len(cp.points)) 414 for s, tableCps := range cp.points { 415 tableCpSnapshots := make(map[string]tablePoint) 416 for tbl, point := range tableCps { 417 if point.outOfDate() { 418 tableCpSnapshots[tbl] = point.savedPoint 419 } 420 } 421 if len(tableCpSnapshots) > 0 { 422 tableCheckPoints[s] = tableCpSnapshots 423 } 424 } 425 426 // flush when 427 // - global checkpoint is forwarded 428 // - global checkpoint is not forwarded but binlog filename updated. This may happen when upstream switched or relay 429 // enable/disable in GTID replication 430 // - the first time to flush checkpoint 431 // - need update safe mode exit point 432 flushGlobalPoint := cp.globalPoint.outOfDate() || 433 cp.globalPoint.savedPoint.location.Position.Name != cp.globalPoint.flushedPoint.location.Position.Name || 434 cp.globalPointSaveTime.IsZero() || 435 (isSyncFlush && cp.needFlushSafeModeExitPoint.Load()) 436 437 // if there is no change on both table points and global point, just return an empty snapshot 438 if len(tableCheckPoints) == 0 && !flushGlobalPoint { 439 return nil 440 } 441 442 snapshot := &remoteCheckpointSnapshot{ 443 id: id, 444 points: tableCheckPoints, 445 } 446 447 globalPoint := &tablePoint{ 448 location: cp.globalPoint.savedPoint.location.Clone(), 449 ti: cp.globalPoint.savedPoint.ti, 450 } 451 if flushGlobalPoint { 452 snapshot.globalPoint = globalPoint 453 snapshot.globalPointSaveTime = time.Now() 454 } 455 456 cp.snapshots = append(cp.snapshots, snapshot) 457 return &SnapshotInfo{ 458 id: id, 459 globalPos: globalPoint.location, 460 } 461 } 462 463 // DiscardPendingSnapshots discard all pending snapshots. 464 func (cp *RemoteCheckPoint) DiscardPendingSnapshots() { 465 cp.Lock() 466 defer cp.Unlock() 467 468 cp.snapshots = nil 469 } 470 471 // Init implements CheckPoint.Init. 472 func (cp *RemoteCheckPoint) Init(tctx *tcontext.Context) (err error) { 473 var db *conn.BaseDB 474 var dbConns []*dbconn.DBConn 475 476 rollbackHolder := fr.NewRollbackHolder("syncer") 477 defer func() { 478 if err != nil { 479 rollbackHolder.RollbackReverseOrder() 480 } 481 }() 482 483 checkPointDB := cp.cfg.To 484 checkPointDB.RawDBCfg = dbconfig.DefaultRawDBConfig().SetReadTimeout(maxCheckPointTimeout) 485 db, dbConns, err = dbconn.CreateConns(tctx, cp.cfg, conn.DownstreamDBConfig(&checkPointDB), 1, cp.cfg.IOTotalBytes, cp.cfg.UUID) 486 if err != nil { 487 return 488 } 489 cp.db = db 490 cp.dbConn = dbConns[0] 491 rollbackHolder.Add(fr.FuncRollback{Name: "CloseRemoteCheckPoint", Fn: cp.Close}) 492 493 err = cp.prepare(tctx) 494 495 return 496 } 497 498 // Close implements CheckPoint.Close. 499 func (cp *RemoteCheckPoint) Close() { 500 dbconn.CloseBaseDB(cp.logCtx, cp.db) 501 } 502 503 // ResetConn implements CheckPoint.ResetConn. 504 func (cp *RemoteCheckPoint) ResetConn(tctx *tcontext.Context) error { 505 return cp.dbConn.ResetConn(tctx) 506 } 507 508 // Clear implements CheckPoint.Clear. 509 func (cp *RemoteCheckPoint) Clear(tctx *tcontext.Context) error { 510 cp.Lock() 511 defer cp.Unlock() 512 513 // delete all checkpoints 514 // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update 515 tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(maxDMLConnectionDuration) 516 defer cancel() 517 _, err := cp.dbConn.ExecuteSQL( 518 tctx2, 519 cp.metricProxies, 520 []string{`DELETE FROM ` + cp.tableName + ` WHERE id = ?`}, 521 []interface{}{cp.id}, 522 ) 523 if err != nil { 524 return err 525 } 526 527 cp.globalPoint = newBinlogPoint(binlog.MustZeroLocation(cp.cfg.Flavor), binlog.MustZeroLocation(cp.cfg.Flavor), nil, nil, cp.cfg.EnableGTID) 528 cp.globalPointSaveTime = time.Time{} 529 cp.lastSnapshotCreationTime = time.Time{} 530 cp.points = make(map[string]map[string]*binlogPoint) 531 cp.snapshots = make([]*remoteCheckpointSnapshot, 0) 532 cp.safeModeExitPoint = nil 533 534 return nil 535 } 536 537 // SaveTablePoint implements CheckPoint.SaveTablePoint. 538 func (cp *RemoteCheckPoint) SaveTablePoint(table *filter.Table, point binlog.Location, ti *model.TableInfo) { 539 cp.Lock() 540 defer cp.Unlock() 541 cp.saveTablePoint(table, point, ti) 542 } 543 544 // saveTablePoint saves single table's checkpoint without mutex.Lock. 545 func (cp *RemoteCheckPoint) saveTablePoint(sourceTable *filter.Table, location binlog.Location, ti *model.TableInfo) { 546 if binlog.CompareLocation(cp.globalPoint.savedPoint.location, location, cp.cfg.EnableGTID) > 0 { 547 panic(fmt.Sprintf("table checkpoint %+v less than global checkpoint %+v", location, cp.globalPoint)) 548 } 549 550 // we save table checkpoint while we meet DDL or DML 551 cp.logCtx.L().Debug("save table checkpoint", zap.Stringer("location", location), zap.Stringer("table", sourceTable)) 552 mSchema, ok := cp.points[sourceTable.Schema] 553 if !ok { 554 mSchema = make(map[string]*binlogPoint) 555 cp.points[sourceTable.Schema] = mSchema 556 } 557 point, ok := mSchema[sourceTable.Name] 558 if !ok { 559 mSchema[sourceTable.Name] = newBinlogPoint(location, binlog.MustZeroLocation(cp.cfg.Flavor), ti, nil, cp.cfg.EnableGTID) 560 } else if err := point.save(location, ti); err != nil { 561 cp.logCtx.L().Error("fail to save table point", zap.Stringer("table", sourceTable), log.ShortError(err)) 562 } 563 } 564 565 // SaveSafeModeExitPoint implements CheckPoint.SaveSafeModeExitPoint 566 // shouldn't call concurrently (only called before loop in Syncer.Run and in loop to reset). 567 func (cp *RemoteCheckPoint) SaveSafeModeExitPoint(point *binlog.Location) { 568 if cp.safeModeExitPoint == nil || point == nil || 569 binlog.CompareLocation(*point, *cp.safeModeExitPoint, cp.cfg.EnableGTID) > 0 { 570 cp.safeModeExitPoint = point 571 cp.needFlushSafeModeExitPoint.Store(true) 572 } 573 } 574 575 // SafeModeExitPoint implements CheckPoint.SafeModeExitPoint. 576 func (cp *RemoteCheckPoint) SafeModeExitPoint() *binlog.Location { 577 return cp.safeModeExitPoint 578 } 579 580 // DeleteTablePoint implements CheckPoint.DeleteTablePoint. 581 func (cp *RemoteCheckPoint) DeleteTablePoint(tctx *tcontext.Context, table *filter.Table) error { 582 cp.Lock() 583 defer cp.Unlock() 584 sourceSchema, sourceTable := table.Schema, table.Name 585 mSchema, ok := cp.points[sourceSchema] 586 if !ok { 587 return nil 588 } 589 _, ok = mSchema[sourceTable] 590 if !ok { 591 return nil 592 } 593 594 // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update 595 tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(maxDMLConnectionDuration) 596 defer cancel() 597 cp.logCtx.L().Info("delete table checkpoint", zap.String("schema", sourceSchema), zap.String("table", sourceTable)) 598 _, err := cp.dbConn.ExecuteSQL( 599 tctx2, 600 cp.metricProxies, 601 []string{`DELETE FROM ` + cp.tableName + ` WHERE id = ? AND cp_schema = ? AND cp_table = ?`}, 602 []interface{}{cp.id, sourceSchema, sourceTable}, 603 ) 604 if err != nil { 605 return err 606 } 607 delete(mSchema, sourceTable) 608 return nil 609 } 610 611 // DeleteAllTablePoint implements CheckPoint.DeleteAllTablePoint. 612 func (cp *RemoteCheckPoint) DeleteAllTablePoint(tctx *tcontext.Context) error { 613 cp.Lock() 614 defer cp.Unlock() 615 616 tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(maxDMLConnectionDuration) 617 defer cancel() 618 cp.logCtx.L().Info("delete all table checkpoint") 619 _, err := cp.dbConn.ExecuteSQL( 620 tctx2, 621 cp.metricProxies, 622 []string{`DELETE FROM ` + cp.tableName + ` WHERE id = ? AND is_global = ?`}, 623 []interface{}{cp.id, false}, 624 ) 625 if err != nil { 626 return err 627 } 628 cp.points = make(map[string]map[string]*binlogPoint) 629 return nil 630 } 631 632 // DeleteSchemaPoint implements CheckPoint.DeleteSchemaPoint. 633 func (cp *RemoteCheckPoint) DeleteSchemaPoint(tctx *tcontext.Context, sourceSchema string) error { 634 cp.Lock() 635 defer cp.Unlock() 636 if _, ok := cp.points[sourceSchema]; !ok { 637 return nil 638 } 639 640 // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update 641 tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(maxDMLConnectionDuration) 642 defer cancel() 643 cp.logCtx.L().Info("delete schema checkpoint", zap.String("schema", sourceSchema)) 644 _, err := cp.dbConn.ExecuteSQL( 645 tctx2, 646 cp.metricProxies, 647 []string{`DELETE FROM ` + cp.tableName + ` WHERE id = ? AND cp_schema = ?`}, 648 []interface{}{cp.id, sourceSchema}, 649 ) 650 if err != nil { 651 return err 652 } 653 delete(cp.points, sourceSchema) 654 return nil 655 } 656 657 // IsOlderThanTablePoint implements CheckPoint.IsOlderThanTablePoint. 658 // This function is used to skip old binlog events. Table checkpoint is saved after dispatching a binlog event. 659 // - For GTID based and position based replication, DML handling is a bit different but comparison is same here. 660 // When using position based, each event has unique position so we have confident to skip event which is <= table checkpoint. 661 // When using GTID based, there may be more than one event with same GTID, but we still skip event which is <= table checkpoint, 662 // to make this right we only save table point for the transaction affected tables only after the whole transaction is processed 663 // - DDL will not have unique position or GTID, so we can always skip events <= table checkpoint. 664 func (cp *RemoteCheckPoint) IsOlderThanTablePoint(table *filter.Table, location binlog.Location) bool { 665 cp.RLock() 666 defer cp.RUnlock() 667 sourceSchema, sourceTable := table.Schema, table.Name 668 mSchema, ok := cp.points[sourceSchema] 669 if !ok { 670 return false 671 } 672 point, ok := mSchema[sourceTable] 673 if !ok { 674 return false 675 } 676 oldLocation := point.MySQLLocation() 677 // if we update enable-gtid = false to true, we need to compare binlog position instead of GTID before we save table point 678 cmpGTID := cp.cfg.EnableGTID && !(gtid.CheckGTIDSetEmpty(oldLocation.GetGTID()) && binlog.ComparePosition(oldLocation.Position, binlog.MinPosition) > 0) 679 cp.logCtx.L().Debug("compare table location whether is newer", zap.Stringer("location", location), zap.Stringer("old location", oldLocation), zap.Bool("cmpGTID", cmpGTID)) 680 681 return binlog.CompareLocation(location, oldLocation, cmpGTID) <= 0 682 } 683 684 // SaveGlobalPoint implements CheckPoint.SaveGlobalPoint. 685 func (cp *RemoteCheckPoint) SaveGlobalPoint(location binlog.Location) { 686 cp.Lock() 687 defer cp.Unlock() 688 689 cp.logCtx.L().Debug("save global checkpoint", zap.Stringer("location", location)) 690 if err := cp.globalPoint.save(location, nil); err != nil { 691 cp.logCtx.L().Error("fail to save global checkpoint", log.ShortError(err)) 692 } 693 } 694 695 // SaveGlobalPointForcibly implements CheckPoint.SaveGlobalPointForcibly. 696 func (cp *RemoteCheckPoint) SaveGlobalPointForcibly(location binlog.Location) { 697 cp.Lock() 698 defer cp.Unlock() 699 700 cp.logCtx.L().Info("reset global checkpoint", zap.Stringer("location", location)) 701 cp.globalPoint = newBinlogPoint(location, binlog.MustZeroLocation(cp.cfg.Flavor), nil, nil, cp.cfg.EnableGTID) 702 } 703 704 // FlushPointsExcept implements CheckPoint.FlushPointsExcept. 705 func (cp *RemoteCheckPoint) FlushPointsExcept( 706 tctx *tcontext.Context, 707 snapshotID int, 708 exceptTables []*filter.Table, 709 extraSQLs []string, 710 extraArgs [][]interface{}, 711 ) error { 712 cp.Lock() 713 714 if len(cp.snapshots) == 0 || cp.snapshots[0].id != snapshotID { 715 return errors.Errorf("snapshot %d not found", snapshotID) 716 } 717 snapshotCp := cp.snapshots[0] 718 cp.snapshots = cp.snapshots[1:] 719 720 // convert slice to map 721 excepts := make(map[string]map[string]struct{}) 722 for _, schemaTable := range exceptTables { 723 schema, table := schemaTable.Schema, schemaTable.Name 724 m, ok := excepts[schema] 725 if !ok { 726 m = make(map[string]struct{}) 727 excepts[schema] = m 728 } 729 m[table] = struct{}{} 730 } 731 732 sqls := make([]string, 0, 100) 733 args := make([][]interface{}, 0, 100) 734 735 type tableCpSnapshotTuple struct { 736 tableCp *binlogPoint // current table checkpoint location 737 snapshotTableCP tablePoint // table checkpoint snapshot location 738 } 739 740 points := make([]*tableCpSnapshotTuple, 0, 100) 741 742 for schema, mSchema := range snapshotCp.points { 743 schemaCp := cp.points[schema] 744 for table, point := range mSchema { 745 if _, ok1 := excepts[schema]; ok1 { 746 if _, ok2 := excepts[schema][table]; ok2 { 747 continue 748 } 749 } 750 tableCP := schemaCp[table] 751 if tableCP.outOfDateBy(point.location) { 752 tiBytes, err := json.Marshal(point.ti) 753 if err != nil { 754 return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, schema, table) 755 } 756 757 sql2, arg := cp.genUpdateSQL(schema, table, point.location, nil, tiBytes, false) 758 sqls = append(sqls, sql2) 759 args = append(args, arg) 760 761 points = append(points, &tableCpSnapshotTuple{ 762 tableCp: tableCP, 763 snapshotTableCP: point, 764 }) 765 } 766 } 767 } 768 for i := range extraSQLs { 769 sqls = append(sqls, extraSQLs[i]) 770 args = append(args, extraArgs[i]) 771 } 772 773 // updating global checkpoint should be the last SQL, its success indicates 774 // the checkpoint is flushed successfully. 775 if snapshotCp.globalPoint != nil { 776 locationG := snapshotCp.globalPoint.location 777 sqlG, argG := cp.genUpdateSQL(globalCpSchema, globalCpTable, locationG, cp.safeModeExitPoint, nil, true) 778 sqls = append(sqls, sqlG) 779 args = append(args, argG) 780 } 781 782 cp.Unlock() 783 784 // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update 785 tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(maxDMLConnectionDuration) 786 defer cancel() 787 err := cp.dbConn.ExecuteSQLAutoSplit(tctx2, cp.metricProxies, sqls, args...) 788 if err != nil { 789 return err 790 } 791 792 if snapshotCp.globalPoint != nil { 793 cp.globalPoint.flushBy(*snapshotCp.globalPoint) 794 cp.Lock() 795 cp.globalPointSaveTime = snapshotCp.globalPointSaveTime 796 cp.Unlock() 797 } 798 799 for _, point := range points { 800 point.tableCp.flushBy(point.snapshotTableCP) 801 } 802 cp.needFlushSafeModeExitPoint.Store(false) 803 return nil 804 } 805 806 // FlushPointsWithTableInfos implements CheckPoint.FlushPointsWithTableInfos. 807 func (cp *RemoteCheckPoint) FlushPointsWithTableInfos(tctx *tcontext.Context, tables []*filter.Table, tis []*model.TableInfo) error { 808 cp.Lock() 809 defer cp.Unlock() 810 // should not happened 811 if len(tables) != len(tis) { 812 return errors.Errorf("the length of the tables is not equal to the length of the table infos, left: %d, right: %d", len(tables), len(tis)) 813 } 814 815 for i := 0; i < len(tables); i += batchFlushPoints { 816 end := i + batchFlushPoints 817 if end > len(tables) { 818 end = len(tables) 819 } 820 821 sqls := make([]string, 0, batchFlushPoints) 822 args := make([][]interface{}, 0, batchFlushPoints) 823 points := make([]*binlogPoint, 0, batchFlushPoints) 824 for j := i; j < end; j++ { 825 table := tables[j] 826 ti := tis[j] 827 sourceSchema, sourceTable := table.Schema, table.Name 828 829 var point *binlogPoint 830 // if point already in memory, use it 831 if tablePoints, ok := cp.points[sourceSchema]; ok { 832 if p, ok2 := tablePoints[sourceTable]; ok2 { 833 point = p 834 } 835 } 836 // create new point 837 if point == nil { 838 cp.saveTablePoint(table, cp.globalPoint.MySQLLocation(), ti) 839 point = cp.points[sourceSchema][sourceTable] 840 } else { 841 point.savedPoint.ti = ti 842 point.flushedPoint.ti = ti 843 } 844 tiBytes, err := json.Marshal(ti) 845 if err != nil { 846 return terror.ErrSchemaTrackerCannotSerialize.Delegate(err, sourceSchema, sourceTable) 847 } 848 location := point.MySQLLocation() 849 sql, arg := cp.genUpdateSQL(sourceSchema, sourceTable, location, nil, tiBytes, false) 850 sqls = append(sqls, sql) 851 args = append(args, arg) 852 points = append(points, point) 853 } 854 // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update 855 tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(conn.DefaultDBTimeout) 856 defer cancel() 857 _, err := cp.dbConn.ExecuteSQL(tctx2, cp.metricProxies, sqls, args...) 858 if err != nil { 859 return err 860 } 861 862 for _, point := range points { 863 point.flush() 864 } 865 } 866 return nil 867 } 868 869 // FlushSafeModeExitPoint implements CheckPoint.FlushSafeModeExitPoint. 870 func (cp *RemoteCheckPoint) FlushSafeModeExitPoint(tctx *tcontext.Context) error { 871 cp.RLock() 872 defer cp.RUnlock() 873 874 sqls := make([]string, 1) 875 args := make([][]interface{}, 1) 876 877 // use FlushedGlobalPoint here to avoid update global checkpoint 878 locationG := cp.FlushedGlobalPoint() 879 sqls[0], args[0] = cp.genUpdateSQL(globalCpSchema, globalCpTable, locationG, cp.safeModeExitPoint, nil, true) 880 881 // use a new context apart from syncer, to make sure when syncer call `cancel` checkpoint could update 882 tctx2, cancel := tctx.WithContext(context.Background()).WithTimeout(maxDMLConnectionDuration) 883 defer cancel() 884 _, err := cp.dbConn.ExecuteSQL(tctx2, cp.metricProxies, sqls, args...) 885 if err != nil { 886 return err 887 } 888 889 cp.needFlushSafeModeExitPoint.Store(false) 890 return nil 891 } 892 893 // GlobalPoint implements CheckPoint.GlobalPoint. 894 func (cp *RemoteCheckPoint) GlobalPoint() binlog.Location { 895 cp.RLock() 896 defer cp.RUnlock() 897 898 return cp.globalPoint.MySQLLocation() 899 } 900 901 // GlobalPointSaveTime implements CheckPoint.GlobalPointSaveTime. 902 func (cp *RemoteCheckPoint) GlobalPointSaveTime() time.Time { 903 cp.RLock() 904 defer cp.RUnlock() 905 906 return cp.globalPointSaveTime 907 } 908 909 // TablePoint implements CheckPoint.TablePoint. 910 func (cp *RemoteCheckPoint) TablePoint() map[string]map[string]binlog.Location { 911 cp.RLock() 912 defer cp.RUnlock() 913 914 tablePoint := make(map[string]map[string]binlog.Location) 915 for schema, tables := range cp.points { 916 tablePoint[schema] = make(map[string]binlog.Location) 917 for table, point := range tables { 918 tablePoint[schema][table] = point.MySQLLocation() 919 } 920 } 921 return tablePoint 922 } 923 924 func (cp *RemoteCheckPoint) GetTableInfo(schema string, table string) *model.TableInfo { 925 cp.RLock() 926 defer cp.RUnlock() 927 928 tables, ok := cp.points[schema] 929 if !ok { 930 return nil 931 } 932 tablePoint, ok := tables[table] 933 if !ok { 934 return nil 935 } 936 return tablePoint.TableInfo() 937 } 938 939 // FlushedGlobalPoint implements CheckPoint.FlushedGlobalPoint. 940 func (cp *RemoteCheckPoint) FlushedGlobalPoint() binlog.Location { 941 cp.RLock() 942 defer cp.RUnlock() 943 944 return cp.globalPoint.FlushedMySQLLocation() 945 } 946 947 // String implements CheckPoint.String. 948 func (cp *RemoteCheckPoint) String() string { 949 cp.RLock() 950 defer cp.RUnlock() 951 952 return cp.globalPoint.String() 953 } 954 955 // LastFlushOutdated implements CheckPoint.LastFlushOutdated. 956 func (cp *RemoteCheckPoint) LastFlushOutdated() bool { 957 cp.RLock() 958 defer cp.RUnlock() 959 960 if time.Since(cp.globalPointSaveTime) < time.Duration(cp.cfg.CheckpointFlushInterval)*time.Second { 961 return false 962 } 963 if time.Since(cp.lastSnapshotCreationTime) < time.Duration(cp.cfg.CheckpointFlushInterval)*time.Second { 964 return false 965 } 966 return true 967 } 968 969 // Rollback implements CheckPoint.Rollback. 970 func (cp *RemoteCheckPoint) Rollback() { 971 cp.RLock() 972 defer cp.RUnlock() 973 cp.globalPoint.rollback() 974 for schemaName, mSchema := range cp.points { 975 for tableName, point := range mSchema { 976 table := &filter.Table{ 977 Schema: schemaName, 978 Name: tableName, 979 } 980 logger := cp.logCtx.L().WithFields(zap.Stringer("table", table)) 981 logger.Debug("try to rollback checkpoint", log.WrapStringerField("checkpoint", point)) 982 point.rollback() 983 } 984 } 985 } 986 987 func (cp *RemoteCheckPoint) prepare(tctx *tcontext.Context) error { 988 if err := cp.createSchema(tctx); err != nil { 989 return err 990 } 991 992 return cp.createTable(tctx) 993 } 994 995 func (cp *RemoteCheckPoint) createSchema(tctx *tcontext.Context) error { 996 // TODO(lance6716): change ColumnName to IdentName or something 997 sql2 := fmt.Sprintf("CREATE SCHEMA IF NOT EXISTS %s", dbutil.ColumnName(cp.cfg.MetaSchema)) 998 args := make([]interface{}, 0) 999 _, err := cp.dbConn.ExecuteSQL(tctx, cp.metricProxies, []string{sql2}, [][]interface{}{args}...) 1000 cp.logCtx.L().Info("create checkpoint schema", zap.String("statement", sql2)) 1001 return err 1002 } 1003 1004 func (cp *RemoteCheckPoint) createTable(tctx *tcontext.Context) error { 1005 sqls := []string{ 1006 `CREATE TABLE IF NOT EXISTS ` + cp.tableName + ` ( 1007 id VARCHAR(32) NOT NULL, 1008 cp_schema VARCHAR(128) NOT NULL, 1009 cp_table VARCHAR(128) NOT NULL, 1010 binlog_name VARCHAR(128), 1011 binlog_pos INT UNSIGNED, 1012 binlog_gtid TEXT, 1013 exit_safe_binlog_name VARCHAR(128) DEFAULT '', 1014 exit_safe_binlog_pos INT UNSIGNED DEFAULT 0, 1015 exit_safe_binlog_gtid TEXT, 1016 table_info JSON NOT NULL, 1017 is_global BOOLEAN, 1018 create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, 1019 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 1020 UNIQUE KEY uk_id_schema_table (id, cp_schema, cp_table) 1021 )`, 1022 } 1023 _, err := cp.dbConn.ExecuteSQL(tctx, cp.metricProxies, sqls) 1024 cp.logCtx.L().Info("create checkpoint table", zap.Strings("statements", sqls)) 1025 return err 1026 } 1027 1028 // Load implements CheckPoint.Load. 1029 func (cp *RemoteCheckPoint) Load(tctx *tcontext.Context) error { 1030 cp.Lock() 1031 defer cp.Unlock() 1032 1033 query := `SELECT cp_schema, cp_table, binlog_name, binlog_pos, binlog_gtid, exit_safe_binlog_name, exit_safe_binlog_pos, exit_safe_binlog_gtid, table_info, is_global FROM ` + cp.tableName + ` WHERE id = ?` 1034 rows, err := cp.dbConn.QuerySQL(tctx, cp.metricProxies, query, cp.id) 1035 defer func() { 1036 if rows != nil { 1037 rows.Close() 1038 } 1039 }() 1040 1041 failpoint.Inject("LoadCheckpointFailed", func(val failpoint.Value) { 1042 err = tmysql.NewErr(uint16(val.(int))) 1043 log.L().Warn("Load failed", zap.String("failpoint", "LoadCheckpointFailed"), zap.Error(err)) 1044 }) 1045 1046 if err != nil { 1047 return terror.WithScope(err, terror.ScopeDownstream) 1048 } 1049 1050 // checkpoints in DB have higher priority 1051 // if don't want to use checkpoint in DB, set `remove-meta` to `true` 1052 var ( 1053 cpSchema string 1054 cpTable string 1055 binlogName string 1056 binlogPos uint32 1057 binlogGTIDSet sql.NullString 1058 exitSafeBinlogName string 1059 exitSafeBinlogPos uint32 1060 exitSafeBinlogGTIDSet sql.NullString 1061 tiBytes []byte 1062 isGlobal bool 1063 ) 1064 for rows.Next() { 1065 err := rows.Scan(&cpSchema, &cpTable, &binlogName, &binlogPos, &binlogGTIDSet, &exitSafeBinlogName, &exitSafeBinlogPos, &exitSafeBinlogGTIDSet, &tiBytes, &isGlobal) 1066 if err != nil { 1067 return terror.DBErrorAdapt(err, cp.dbConn.Scope(), terror.ErrDBDriverError) 1068 } 1069 1070 gset, err := gtid.ParserGTID(cp.cfg.Flavor, binlogGTIDSet.String) // default to "". 1071 if err != nil { 1072 return err 1073 } 1074 1075 location := binlog.NewLocation( 1076 mysql.Position{ 1077 Name: binlogName, 1078 Pos: binlogPos, 1079 }, 1080 gset, 1081 ) 1082 if isGlobal { 1083 // Use IsFreshPosition here to make sure checkpoint can be updated if gset is empty 1084 if !binlog.IsFreshPosition(location, cp.cfg.Flavor, cp.cfg.EnableGTID) { 1085 cp.globalPoint = newBinlogPoint(location, location, nil, nil, cp.cfg.EnableGTID) 1086 cp.logCtx.L().Info("fetch global checkpoint from DB", log.WrapStringerField("global checkpoint", cp.globalPoint)) 1087 } 1088 1089 if cp.cfg.EnableGTID { 1090 // gtid set default is "", but upgrade may cause NULL value 1091 if exitSafeBinlogGTIDSet.Valid && exitSafeBinlogGTIDSet.String != "" { 1092 gset2, err2 := gtid.ParserGTID(cp.cfg.Flavor, exitSafeBinlogGTIDSet.String) 1093 if err2 != nil { 1094 return err2 1095 } 1096 exitSafeModeLoc := binlog.NewLocation( 1097 mysql.Position{ 1098 Name: exitSafeBinlogName, 1099 Pos: exitSafeBinlogPos, 1100 }, 1101 gset2, 1102 ) 1103 cp.SaveSafeModeExitPoint(&exitSafeModeLoc) 1104 } // TODO: we forget to handle else... 1105 } else { 1106 if exitSafeBinlogName != "" { 1107 exitSafeModeLoc := binlog.Location{ 1108 Position: mysql.Position{ 1109 Name: exitSafeBinlogName, 1110 Pos: exitSafeBinlogPos, 1111 }, 1112 } 1113 cp.SaveSafeModeExitPoint(&exitSafeModeLoc) 1114 } 1115 } 1116 continue // skip global checkpoint 1117 } 1118 1119 var ti *model.TableInfo 1120 if !bytes.Equal(tiBytes, []byte("null")) { 1121 // only create table if `table_info` is not `null`. 1122 if err = json.Unmarshal(tiBytes, &ti); err != nil { 1123 return terror.ErrSchemaTrackerInvalidJSON.Delegate(err, cpSchema, cpTable) 1124 } 1125 } 1126 1127 mSchema, ok := cp.points[cpSchema] 1128 if !ok { 1129 mSchema = make(map[string]*binlogPoint) 1130 cp.points[cpSchema] = mSchema 1131 } 1132 mSchema[cpTable] = newBinlogPoint(location, location, ti, ti, cp.cfg.EnableGTID) 1133 } 1134 1135 return terror.DBErrorAdapt(rows.Err(), cp.dbConn.Scope(), terror.ErrDBDriverError) 1136 } 1137 1138 // LoadIntoSchemaTracker loads table infos of all points into schema tracker. 1139 func (cp *RemoteCheckPoint) LoadIntoSchemaTracker(ctx context.Context, schemaTracker *schema.Tracker) error { 1140 cp.RLock() 1141 defer cp.RUnlock() 1142 1143 tablesToCreate := map[string]map[string]*model.TableInfo{} 1144 for cpSchema, mSchema := range cp.points { 1145 for cpTable, point := range mSchema { 1146 // for create database DDL, we'll create a table point with no table name and table info, need to skip. 1147 if point.flushedPoint.ti == nil { 1148 continue 1149 } 1150 if _, ok := tablesToCreate[cpSchema]; !ok { 1151 tablesToCreate[cpSchema] = make(map[string]*model.TableInfo) 1152 } 1153 tablesToCreate[cpSchema][cpTable] = point.flushedPoint.ti 1154 cp.logCtx.L().Debug("will init table info in schema tracker", 1155 zap.String("database", cpSchema), 1156 zap.String("table", cpTable)) 1157 } 1158 } 1159 return schemaTracker.BatchCreateTableIfNotExist(tablesToCreate) 1160 } 1161 1162 // CheckAndUpdate check the checkpoint data consistency and try to fix them if possible. 1163 func (cp *RemoteCheckPoint) CheckAndUpdate(ctx context.Context, schemas map[string]string, tables map[string]map[string]string) error { 1164 cp.Lock() 1165 hasChange := false 1166 for lcSchema, tableMap := range tables { 1167 tableCps, ok := cp.points[lcSchema] 1168 if !ok { 1169 continue 1170 } 1171 for lcTable, table := range tableMap { 1172 tableCp, ok := tableCps[lcTable] 1173 if !ok { 1174 continue 1175 } 1176 tableCps[table] = tableCp 1177 delete(tableCps, lcTable) 1178 hasChange = true 1179 } 1180 } 1181 for lcSchema, schema := range schemas { 1182 if tableCps, ok := cp.points[lcSchema]; ok { 1183 cp.points[schema] = tableCps 1184 delete(cp.points, lcSchema) 1185 hasChange = true 1186 } 1187 } 1188 cp.Unlock() 1189 1190 if hasChange { 1191 tctx := cp.logCtx.WithContext(ctx) 1192 cpID := cp.Snapshot(true) 1193 if cpID != nil { 1194 return cp.FlushPointsExcept(tctx, cpID.id, nil, nil, nil) 1195 } 1196 } 1197 return nil 1198 } 1199 1200 // LoadMeta implements CheckPoint.LoadMeta. 1201 func (cp *RemoteCheckPoint) LoadMeta(ctx context.Context) error { 1202 cp.Lock() 1203 defer cp.Unlock() 1204 1205 var ( 1206 location *binlog.Location 1207 safeModeExitLoc *binlog.Location 1208 err error 1209 ) 1210 switch cp.cfg.Mode { 1211 case config.ModeAll, config.ModeLoadSync: 1212 // NOTE: syncer must continue the syncing follow loader's tail, so we parse mydumper's output 1213 // refine when master / slave switching added and checkpoint mechanism refactored 1214 location, safeModeExitLoc, err = cp.parseMetaData(ctx) 1215 if err != nil { 1216 return err 1217 } 1218 case config.ModeIncrement: 1219 // load meta from task config 1220 if cp.cfg.Meta == nil { 1221 cp.logCtx.L().Warn("didn't set meta in increment task-mode") 1222 cp.globalPoint = newBinlogPoint(binlog.MustZeroLocation(cp.cfg.Flavor), binlog.MustZeroLocation(cp.cfg.Flavor), nil, nil, cp.cfg.EnableGTID) 1223 return nil 1224 } 1225 gset, err := gtid.ParserGTID(cp.cfg.Flavor, cp.cfg.Meta.BinLogGTID) 1226 if err != nil { 1227 return err 1228 } 1229 1230 loc := binlog.NewLocation( 1231 mysql.Position{ 1232 Name: cp.cfg.Meta.BinLogName, 1233 Pos: cp.cfg.Meta.BinLogPos, 1234 }, 1235 gset, 1236 ) 1237 location = &loc 1238 default: 1239 // should not go here (syncer is only used in `all` or `incremental` mode) 1240 return terror.ErrCheckpointInvalidTaskMode.Generate(cp.cfg.Mode) 1241 } 1242 1243 // if meta loaded, we will start syncing from meta's pos 1244 if location != nil { 1245 cp.globalPoint = newBinlogPoint(*location, *location, nil, nil, cp.cfg.EnableGTID) 1246 cp.logCtx.L().Info("loaded checkpoints from meta", log.WrapStringerField("global checkpoint", cp.globalPoint)) 1247 } 1248 if safeModeExitLoc != nil { 1249 cp.SaveSafeModeExitPoint(safeModeExitLoc) 1250 cp.logCtx.L().Info("set SafeModeExitLoc from meta", zap.Stringer("SafeModeExitLoc", safeModeExitLoc)) 1251 } 1252 1253 return nil 1254 } 1255 1256 // genUpdateSQL generates SQL and arguments for update checkpoint. 1257 func (cp *RemoteCheckPoint) genUpdateSQL(cpSchema, cpTable string, location binlog.Location, safeModeExitLoc *binlog.Location, tiBytes []byte, isGlobal bool) (string, []interface{}) { 1258 // use `INSERT INTO ... ON DUPLICATE KEY UPDATE` rather than `REPLACE INTO` 1259 // to keep `create_time`, `update_time` correctly 1260 sql2 := `INSERT INTO ` + cp.tableName + ` 1261 (id, cp_schema, cp_table, binlog_name, binlog_pos, binlog_gtid, exit_safe_binlog_name, exit_safe_binlog_pos, exit_safe_binlog_gtid, table_info, is_global) VALUES 1262 (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?) 1263 ON DUPLICATE KEY UPDATE 1264 binlog_name = VALUES(binlog_name), 1265 binlog_pos = VALUES(binlog_pos), 1266 binlog_gtid = VALUES(binlog_gtid), 1267 exit_safe_binlog_name = VALUES(exit_safe_binlog_name), 1268 exit_safe_binlog_pos = VALUES(exit_safe_binlog_pos), 1269 exit_safe_binlog_gtid = VALUES(exit_safe_binlog_gtid), 1270 table_info = VALUES(table_info), 1271 is_global = VALUES(is_global); 1272 ` 1273 1274 if isGlobal { 1275 cpSchema = globalCpSchema 1276 cpTable = globalCpTable 1277 } 1278 1279 if len(tiBytes) == 0 { 1280 tiBytes = []byte("null") 1281 } 1282 1283 var ( 1284 exitSafeName string 1285 exitSafePos uint32 1286 exitSafeGTIDStr string 1287 ) 1288 if safeModeExitLoc != nil { 1289 exitSafeName = safeModeExitLoc.Position.Name 1290 exitSafePos = safeModeExitLoc.Position.Pos 1291 exitSafeGTIDStr = safeModeExitLoc.GTIDSetStr() 1292 } 1293 1294 // convert tiBytes to string to get a readable log 1295 args := []interface{}{ 1296 cp.id, cpSchema, cpTable, location.Position.Name, location.Position.Pos, location.GTIDSetStr(), 1297 exitSafeName, exitSafePos, exitSafeGTIDStr, string(tiBytes), isGlobal, 1298 } 1299 return sql2, args 1300 } 1301 1302 func (cp *RemoteCheckPoint) parseMetaData(ctx context.Context) (*binlog.Location, *binlog.Location, error) { 1303 // `metadata` is mydumper's output meta file name 1304 filename := "metadata" 1305 loc, loc2, err := dumpling.ParseMetaData(ctx, cp.cfg.LoaderConfig.Dir, filename, cp.cfg.ExtStorage) 1306 if err != nil { 1307 toPrint, err2 := storage.ReadFile(ctx, cp.cfg.LoaderConfig.Dir, filename, nil) 1308 if err2 != nil { 1309 toPrint = []byte(err2.Error()) 1310 } 1311 err = terror.ErrParseMydumperMeta.Generate(err, toPrint) 1312 } 1313 1314 return loc, loc2, err 1315 }