github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/validator_checkpoint.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 syncer 15 16 import ( 17 "database/sql" 18 "encoding/json" 19 "fmt" 20 "strconv" 21 "strings" 22 "time" 23 24 "github.com/go-mysql-org/go-mysql/mysql" 25 "github.com/pingcap/errors" 26 "github.com/pingcap/failpoint" 27 "github.com/pingcap/tidb/pkg/util/dbutil" 28 "github.com/pingcap/tidb/pkg/util/filter" 29 "github.com/pingcap/tiflow/dm/config" 30 "github.com/pingcap/tiflow/dm/pb" 31 "github.com/pingcap/tiflow/dm/pkg/binlog" 32 "github.com/pingcap/tiflow/dm/pkg/conn" 33 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 34 "github.com/pingcap/tiflow/dm/pkg/cputil" 35 "github.com/pingcap/tiflow/dm/pkg/gtid" 36 "github.com/pingcap/tiflow/dm/pkg/log" 37 "github.com/pingcap/tiflow/dm/pkg/retry" 38 "go.uber.org/atomic" 39 "go.uber.org/zap" 40 ) 41 42 const ( 43 maxRowKeyLength = 64 44 45 validationDBTimeout = queryTimeout * 5 46 ) 47 48 var mapErrType2Str = map[validateFailedType]string{ 49 deletedRowExists: "Deleted rows exist", 50 rowNotExist: "Expected rows not exist", 51 rowDifferent: "Column data not matched", 52 } 53 54 var maxRowKeyLengthStr = strconv.Itoa(maxRowKeyLength) 55 56 type validatorPersistHelper struct { 57 L log.Logger 58 cfg *config.SubTaskConfig 59 validator *DataValidator 60 retryer *retry.FiniteRetryer 61 62 checkpointTableName string 63 pendingChangeTableName string 64 errorChangeTableName string 65 tableStatusTableName string 66 67 db *conn.BaseDB 68 schemaInitialized atomic.Bool 69 revision int64 70 } 71 72 func newValidatorCheckpointHelper(validator *DataValidator) *validatorPersistHelper { 73 cfg := validator.cfg 74 logger := validator.L 75 retryer := &retry.FiniteRetryer{ 76 Params: retry.NewParams(3, 5*time.Second, retry.LinearIncrease, 77 func(i int, err error) bool { 78 logger.Warn("met error", zap.Error(err)) 79 return isRetryableDBError(err) 80 }, 81 ), 82 } 83 c := &validatorPersistHelper{ 84 L: logger, 85 cfg: cfg, 86 validator: validator, 87 retryer: retryer, 88 89 checkpointTableName: dbutil.TableName(cfg.MetaSchema, cputil.ValidatorCheckpoint(cfg.Name)), 90 pendingChangeTableName: dbutil.TableName(cfg.MetaSchema, cputil.ValidatorPendingChange(cfg.Name)), 91 errorChangeTableName: dbutil.TableName(cfg.MetaSchema, cputil.ValidatorErrorChange(cfg.Name)), 92 tableStatusTableName: dbutil.TableName(cfg.MetaSchema, cputil.ValidatorTableStatus(cfg.Name)), 93 } 94 95 return c 96 } 97 98 func (c *validatorPersistHelper) init(tctx *tcontext.Context) error { 99 c.db = c.validator.toDB 100 101 if !c.schemaInitialized.Load() { 102 workFunc := func(tctx *tcontext.Context) (interface{}, error) { 103 return nil, c.createSchemaAndTables(tctx) 104 } 105 if _, cnt, err := c.retryer.Apply(tctx, workFunc); err != nil { 106 tctx.L().Error("failed to init validator helper after retry", 107 zap.Int("retry-times", cnt), zap.Error(err)) 108 return err 109 } 110 111 c.schemaInitialized.Store(true) 112 } 113 return nil 114 } 115 116 func (c *validatorPersistHelper) createSchemaAndTables(tctx *tcontext.Context) error { 117 if err := c.createSchema(tctx); err != nil { 118 return err 119 } 120 return c.createTable(tctx) 121 } 122 123 func (c *validatorPersistHelper) createSchema(tctx *tcontext.Context) error { 124 query := fmt.Sprintf("CREATE SCHEMA IF NOT EXISTS %s", dbutil.ColumnName(c.cfg.MetaSchema)) 125 _, err := c.db.ExecContext(tctx, query) 126 tctx.L().Info("create checkpoint schema", zap.String("statement", query)) 127 return err 128 } 129 130 func (c *validatorPersistHelper) createTable(tctx *tcontext.Context) error { 131 sqls := []string{ 132 `CREATE TABLE IF NOT EXISTS ` + c.checkpointTableName + ` ( 133 id BIGINT NOT NULL AUTO_INCREMENT PRIMARY KEY, 134 source VARCHAR(32) NOT NULL, 135 binlog_name VARCHAR(128), 136 binlog_pos INT UNSIGNED, 137 binlog_gtid TEXT, 138 procd_ins BIGINT UNSIGNED NOT NULL, 139 procd_upd BIGINT UNSIGNED NOT NULL, 140 procd_del BIGINT UNSIGNED NOT NULL, 141 revision bigint NOT NULL, 142 create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, 143 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 144 UNIQUE KEY uk_source (source) 145 )`, 146 `CREATE TABLE IF NOT EXISTS ` + c.pendingChangeTableName + ` ( 147 id BIGINT NOT NULL AUTO_INCREMENT PRIMARY KEY, 148 source VARCHAR(32) NOT NULL, 149 schema_name VARCHAR(128) NOT NULL, 150 table_name VARCHAR(128) NOT NULL, 151 row_pk VARCHAR(` + maxRowKeyLengthStr + `) NOT NULL, 152 data JSON NOT NULL, 153 revision bigint NOT NULL, 154 create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, 155 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 156 INDEX idx_source_schema_table_key(source, schema_name, table_name, row_pk), 157 INDEX idx_revision(revision) 158 )`, 159 `CREATE TABLE IF NOT EXISTS ` + c.errorChangeTableName + ` ( 160 id BIGINT NOT NULL AUTO_INCREMENT PRIMARY KEY, 161 source VARCHAR(32) NOT NULL, 162 src_schema_name VARCHAR(128) NOT NULL, 163 src_table_name VARCHAR(128) NOT NULL, 164 row_pk VARCHAR(` + maxRowKeyLengthStr + `) NOT NULL, 165 dst_schema_name VARCHAR(128) NOT NULL, 166 dst_table_name VARCHAR(128) NOT NULL, 167 data JSON NOT NULL, 168 dst_data JSON NOT NULL, 169 error_type int NOT NULL, 170 status int NOT NULL, 171 create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, 172 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 173 UNIQUE KEY uk_source_schema_table_key(source, src_schema_name, src_table_name, row_pk), 174 INDEX idx_status(status) 175 )`, 176 `CREATE TABLE IF NOT EXISTS ` + c.tableStatusTableName + ` ( 177 id BIGINT NOT NULL AUTO_INCREMENT PRIMARY KEY, 178 source VARCHAR(32) NOT NULL, 179 src_schema_name VARCHAR(128) NOT NULL, 180 src_table_name VARCHAR(128) NOT NULL, 181 dst_schema_name VARCHAR(128) NOT NULL, 182 dst_table_name VARCHAR(128) NOT NULL, 183 stage VARCHAR(32) NOT NULL, 184 message VARCHAR(512) NOT NULL, 185 create_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, 186 update_time timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP, 187 UNIQUE KEY uk_source_schema_table_key(source, src_schema_name, src_table_name), 188 INDEX idx_stage(stage) 189 )`, 190 } 191 tctx.L().Info("create checkpoint and data table", zap.Strings("statements", sqls)) 192 for _, q := range sqls { 193 if _, err := c.db.ExecContext(tctx, q); err != nil { 194 return err 195 } 196 } 197 return nil 198 } 199 200 type tableChangeDataForPersist struct { 201 sourceTable *filter.Table 202 columnCount int 203 rows map[string]*rowChangeDataForPersist 204 } 205 206 type rowChangeDataForPersist struct { 207 Key string `json:"key"` 208 Tp rowChangeJobType `json:"tp"` 209 Size int32 `json:"size"` 210 Data []interface{} `json:"data"` 211 FailedCnt int `json:"failed-cnt"` // failed count 212 } 213 214 var triggeredFailOnPersistForIntegrationTest bool 215 216 func (c *validatorPersistHelper) execQueriesWithRetry(tctx *tcontext.Context, queries []string, args [][]interface{}) error { 217 workFunc := func(tctx *tcontext.Context) (interface{}, error) { 218 for i, q := range queries { 219 failpoint.Inject("ValidatorFailOnPersist", func() { 220 // on persist pending row changes, the queries would be [delete, insert...] 221 // if there are 5 inserts, we fail for one time 222 // for source mysql-replica-01, fail on the 3rd 223 // for source mysql-replica-02, fail on the 4th 224 if strings.Contains(q, "_validator_pending_change") && len(queries) == 6 && 225 !triggeredFailOnPersistForIntegrationTest { 226 if (c.cfg.SourceID == "mysql-replica-01" && i == 3) || 227 (c.cfg.SourceID == "mysql-replica-02" && i == 4) { 228 triggeredFailOnPersistForIntegrationTest = true 229 // "Error 1406" is non-resumable error, so we can't retry it 230 failpoint.Return(nil, errors.New("ValidatorFailOnPersist Error 1406")) 231 } 232 } 233 }) 234 if _, err := c.db.ExecContext(tctx, q, args[i]...); err != nil { 235 return nil, err 236 } 237 } 238 return nil, nil 239 } 240 _, _, err2 := c.retryer.Apply(tctx, workFunc) 241 return err2 242 } 243 244 func (c *validatorPersistHelper) persistTableStatusAndErrors(tctx *tcontext.Context) error { 245 // get snapshot of the current table status 246 tableStatus := c.validator.getTableStatusMap() 247 count := len(tableStatus) + int(c.validator.getNewErrorRowCount()) 248 queries := make([]string, 0, count) 249 args := make([][]interface{}, 0, count) 250 251 // upsert table status 252 for _, state := range tableStatus { 253 query := `INSERT INTO ` + c.tableStatusTableName + ` 254 (source, src_schema_name, src_table_name, dst_schema_name, dst_table_name, stage, message) 255 VALUES (?, ?, ?, ?, ?, ?, ?) ON DUPLICATE KEY UPDATE 256 source = VALUES(source), 257 src_schema_name = VALUES(src_schema_name), 258 src_table_name = VALUES(src_table_name), 259 dst_schema_name = VALUES(dst_schema_name), 260 dst_table_name = VALUES(dst_table_name), 261 stage = VALUES(stage), 262 message = VALUES(message) 263 ` 264 queries = append(queries, query) 265 args = append(args, []interface{}{ 266 c.cfg.SourceID, state.source.Schema, state.source.Name, state.target.Schema, state.target.Name, 267 int(state.stage), state.message, 268 }, 269 ) 270 } 271 // upsert error rows 272 for _, worker := range c.validator.getWorkers() { 273 for _, r := range worker.getErrorRows() { 274 query := `INSERT INTO ` + c.errorChangeTableName + ` 275 (source, src_schema_name, src_table_name, row_pk, dst_schema_name, dst_table_name, data, dst_data, error_type, status) 276 VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?) ON DUPLICATE KEY UPDATE 277 source = VALUES(source), 278 src_schema_name = VALUES(src_schema_name), 279 src_table_name = VALUES(src_table_name), 280 row_pk = VALUES(row_pk), 281 dst_schema_name = VALUES(dst_schema_name), 282 dst_table_name = VALUES(dst_table_name), 283 data = VALUES(data), 284 dst_data = VALUES(dst_data), 285 error_type = VALUES(error_type), 286 status = VALUES(status) 287 ` 288 queries = append(queries, query) 289 290 row := r.srcJob.row 291 srcDataBytes, err := json.Marshal(row.RowValues()) 292 if err != nil { 293 return err 294 } 295 dstData := make([]interface{}, len(r.dstData)) 296 for i, d := range r.dstData { 297 if d.Valid { 298 dstData[i] = d.String 299 } 300 } 301 dstDataBytes, err := json.Marshal(dstData) 302 if err != nil { 303 return err 304 } 305 sourceTable := row.GetSourceTable() 306 targetTable := row.GetTargetTable() 307 args = append(args, []interface{}{ 308 c.cfg.SourceID, sourceTable.Schema, sourceTable.Table, r.srcJob.Key, 309 targetTable.Schema, targetTable.Table, 310 string(srcDataBytes), string(dstDataBytes), r.tp, pb.ValidateErrorState_NewErr, 311 }) 312 } 313 } 314 315 return c.execQueriesWithRetry(tctx, queries, args) 316 } 317 318 func (c *validatorPersistHelper) persistPendingRows(tctx *tcontext.Context, rev int64) error { 319 count := int(c.validator.getAllPendingRowCount()) + 1 320 queries := make([]string, 0, count) 321 args := make([][]interface{}, 0, count) 322 323 // delete pending rows left by previous failed call of "persist" 324 queries = append(queries, `DELETE FROM `+c.pendingChangeTableName+` WHERE source = ? and revision = ?`) 325 args = append(args, []interface{}{c.cfg.SourceID, rev}) 326 // insert pending row changes with revision=rev 327 for _, worker := range c.validator.getWorkers() { 328 for _, tblChange := range worker.getPendingChangesMap() { 329 for key, j := range tblChange.jobs { 330 row := j.row 331 // we don't store FirstValidateTS into meta 332 rowForPersist := rowChangeDataForPersist{ 333 Key: key, 334 Tp: j.Tp, 335 Size: j.size, 336 Data: row.RowValues(), 337 FailedCnt: j.FailedCnt, 338 } 339 rowJSON, err := json.Marshal(&rowForPersist) 340 if err != nil { 341 return err 342 } 343 query := `INSERT INTO ` + c.pendingChangeTableName + ` 344 (source, schema_name, table_name, row_pk, data, revision) VALUES (?, ?, ?, ?, ?, ?)` 345 queries = append(queries, query) 346 sourceTable := row.GetSourceTable() 347 args = append(args, []interface{}{ 348 c.cfg.SourceID, 349 sourceTable.Schema, 350 sourceTable.Table, 351 key, 352 string(rowJSON), 353 rev, 354 }) 355 } 356 } 357 } 358 return c.execQueriesWithRetry(tctx, queries, args) 359 } 360 361 func (c *validatorPersistHelper) persist(tctx *tcontext.Context, loc binlog.Location) error { 362 newCtx, cancelFunc := tctx.WithTimeout(validationDBTimeout) 363 defer cancelFunc() 364 // we run sql one by one to avoid potential "transaction too large" error since pending data may quite large. 365 // we use "upsert" to save table status and error row changes, if error happens when persist checkpoint and 366 // pending data, those data maybe inconsistent with each other. but it doesn't matter in validation case, 367 // since the status of the table will be in correct stage after resume and reach the location again. 368 if err := c.persistTableStatusAndErrors(newCtx); err != nil { 369 return err 370 } 371 372 nextRevision := c.revision + 1 373 374 // we use "insert" to save pending data to speed up(see https://asktug.com/t/topic/33147) since 375 // the number of pending rows maybe large. 376 // we run sql one by one to avoid potential "transaction too large" error since pending data may quite large. 377 // And use revision field to associate checkpoint table and pending row table 378 if err := c.persistPendingRows(newCtx, nextRevision); err != nil { 379 return err 380 } 381 382 // upsert checkpoint 383 query := `INSERT INTO ` + c.checkpointTableName + 384 `(source, binlog_name, binlog_pos, binlog_gtid, procd_ins, procd_upd, procd_del, revision) 385 VALUES (?, ?, ?, ?, ?, ?, ?, ?) ON DUPLICATE KEY UPDATE 386 source = VALUES(source), 387 binlog_name = VALUES(binlog_name), 388 binlog_pos = VALUES(binlog_pos), 389 binlog_gtid = VALUES(binlog_gtid), 390 procd_ins = VALUES(procd_ins), 391 procd_upd = VALUES(procd_upd), 392 procd_del = VALUES(procd_del), 393 revision = VALUES(revision) 394 ` 395 rowCounts := c.validator.getProcessedRowCounts() 396 args := []interface{}{ 397 c.cfg.SourceID, loc.Position.Name, loc.Position.Pos, loc.GTIDSetStr(), 398 rowCounts[rowInsert], rowCounts[rowUpdated], rowCounts[rowDeleted], 399 nextRevision, 400 } 401 if err := c.execQueriesWithRetry(newCtx, []string{query}, [][]interface{}{args}); err != nil { 402 return err 403 } 404 405 // if we reach here, checkpoint with new revision is persisted successfully, 406 // but we need to clean up previous pending row changes, i.e. rows with different revision. 407 // it's ok to fail here, next persist will try to delete again, so just log it. 408 query = `DELETE FROM ` + c.pendingChangeTableName + ` WHERE source = ? and revision != ?` 409 args = []interface{}{c.cfg.SourceID, nextRevision} 410 if err := c.execQueriesWithRetry(newCtx, []string{query}, [][]interface{}{args}); err != nil { 411 c.L.Warn("failed to delete previous pending row changes", zap.Error(err), zap.Reflect("args", args)) 412 // nolint:nilerr 413 } 414 415 // to next revision 416 c.revision++ 417 418 return nil 419 } 420 421 type persistedData struct { 422 checkpoint *binlog.Location 423 processedRowCounts []int64 424 pendingChanges map[string]*tableChangeDataForPersist // key is full name of source table 425 rev int64 426 tableStatus map[string]*tableValidateStatus // key is full name of source table 427 } 428 429 func (c *validatorPersistHelper) loadPersistedDataRetry(tctx *tcontext.Context) (*persistedData, error) { 430 start := time.Now() 431 newCtx, cancelFunc := tctx.WithTimeout(validationDBTimeout) 432 defer cancelFunc() 433 workFunc := func(tctx *tcontext.Context) (interface{}, error) { 434 return c.loadPersistedData(tctx) 435 } 436 ret, i, err := c.retryer.Apply(newCtx, workFunc) 437 if err != nil { 438 c.L.Error("failed load persisted data after retry", zap.Int("retry-times", i), zap.Error(err)) 439 return nil, err 440 } 441 442 c.L.Info("loaded persisted data", zap.Duration("time taken", time.Since(start))) 443 return ret.(*persistedData), nil 444 } 445 446 func (c *validatorPersistHelper) loadPersistedData(tctx *tcontext.Context) (*persistedData, error) { 447 var err error 448 data := &persistedData{} 449 data.checkpoint, data.processedRowCounts, data.rev, err = c.loadCheckpoint(tctx) 450 if err != nil { 451 return data, err 452 } 453 454 data.pendingChanges, err = c.loadPendingChange(tctx, data.rev) 455 if err != nil { 456 return data, err 457 } 458 459 data.tableStatus, err = c.loadTableStatus(tctx) 460 if err != nil { 461 return data, err 462 } 463 464 return data, nil 465 } 466 467 func (c *validatorPersistHelper) loadCheckpoint(tctx *tcontext.Context) (*binlog.Location, []int64, int64, error) { 468 start := time.Now() 469 query := `select 470 binlog_name, binlog_pos, binlog_gtid, 471 procd_ins, procd_upd, procd_del, 472 revision 473 from ` + c.checkpointTableName + ` where source = ?` 474 rows, err := c.db.QueryContext(tctx, query, c.cfg.SourceID) 475 if err != nil { 476 return nil, nil, 0, err 477 } 478 defer rows.Close() 479 480 var location *binlog.Location 481 processRowCounts := make([]int64, rowChangeTypeCount) 482 483 // at most one row 484 var revision int64 485 if rows.Next() { 486 var ( 487 binlogName, binlogGtidStr string 488 ins, upd, del int64 489 binlogPos uint32 490 ) 491 492 err = rows.Scan(&binlogName, &binlogPos, &binlogGtidStr, &ins, &upd, &del, &revision) 493 if err != nil { 494 return nil, nil, 0, err 495 } 496 gset, err2 := gtid.ParserGTID(c.cfg.Flavor, binlogGtidStr) 497 if err2 != nil { 498 return nil, nil, 0, err2 499 } 500 tmpLoc := binlog.NewLocation(mysql.Position{Name: binlogName, Pos: binlogPos}, gset) 501 location = &tmpLoc 502 processRowCounts = []int64{ins, upd, del} 503 } 504 if err = rows.Err(); err != nil { 505 return nil, nil, 0, err 506 } 507 c.L.Info("checkpoint loaded", zap.Reflect("loc", location), 508 zap.Int64s("processed(i, u, d)", processRowCounts), zap.Int64("rev", revision), 509 zap.Duration("time taken", time.Since(start))) 510 return location, processRowCounts, revision, nil 511 } 512 513 func (c *validatorPersistHelper) loadPendingChange(tctx *tcontext.Context, rev int64) (map[string]*tableChangeDataForPersist, error) { 514 start := time.Now() 515 res := make(map[string]*tableChangeDataForPersist) 516 query := "select schema_name, table_name, row_pk, data, revision from " + c.pendingChangeTableName + 517 " where source = ? and revision = ?" 518 rows, err := c.db.QueryContext(tctx, query, c.cfg.SourceID, rev) 519 if err != nil { 520 return nil, err 521 } 522 defer rows.Close() 523 524 var count int 525 for rows.Next() { 526 var ( 527 schemaName, tableName, key string 528 data []byte 529 revision int64 530 ) 531 err = rows.Scan(&schemaName, &tableName, &key, &data, &revision) 532 if err != nil { 533 return nil, err 534 } 535 var row *rowChangeDataForPersist 536 err = json.Unmarshal(data, &row) 537 if err != nil { 538 return nil, err 539 } 540 541 sourceTbl := filter.Table{Schema: schemaName, Name: tableName} 542 fullTableName := sourceTbl.String() 543 tblChange, ok := res[fullTableName] 544 if !ok { 545 tblChange = &tableChangeDataForPersist{ 546 sourceTable: &sourceTbl, 547 columnCount: len(row.Data), 548 rows: make(map[string]*rowChangeDataForPersist), 549 } 550 res[fullTableName] = tblChange 551 } 552 tblChange.rows[key] = row 553 rev = revision 554 count++ 555 } 556 557 if err = rows.Err(); err != nil { 558 return nil, err 559 } 560 c.L.Info("pending change loaded", zap.Reflect("count", count), zap.Reflect("rev", rev), 561 zap.Duration("time taken", time.Since(start))) 562 return res, nil 563 } 564 565 func (c *validatorPersistHelper) loadTableStatus(tctx *tcontext.Context) (map[string]*tableValidateStatus, error) { 566 start := time.Now() 567 res := make(map[string]*tableValidateStatus) 568 query := "select src_schema_name, src_table_name, dst_schema_name, dst_table_name, stage, message from " + 569 c.tableStatusTableName + " where source = ?" 570 rows, err := c.db.QueryContext(tctx, query, c.cfg.SourceID) 571 if err != nil { 572 return nil, err 573 } 574 defer rows.Close() 575 576 for rows.Next() { 577 var ( 578 srcSchemaName, srcTableName, dstSchemaName, dstTableName string 579 stage int 580 message string 581 ) 582 err = rows.Scan(&srcSchemaName, &srcTableName, &dstSchemaName, &dstTableName, &stage, &message) 583 if err != nil { 584 return nil, err 585 } 586 srcTbl := filter.Table{Schema: srcSchemaName, Name: srcTableName} 587 fullTableName := srcTbl.String() 588 res[fullTableName] = &tableValidateStatus{ 589 source: srcTbl, 590 target: filter.Table{Schema: dstSchemaName, Name: dstTableName}, 591 stage: pb.Stage(stage), 592 message: message, 593 } 594 } 595 596 if err = rows.Err(); err != nil { 597 return nil, err 598 } 599 c.L.Info("table status loaded", zap.Reflect("count", len(res)), 600 zap.Duration("time taken", time.Since(start))) 601 return res, nil 602 } 603 604 func (c *validatorPersistHelper) loadErrorCount(tctx *tcontext.Context, db *conn.BaseDB) (map[pb.ValidateErrorState]int64, error) { 605 res := make(map[pb.ValidateErrorState]int64) 606 query := "select status, count(*) from " + c.errorChangeTableName + " where source = ? group by status" 607 rows, err := db.QueryContext(tctx, query, c.cfg.SourceID) 608 if err != nil { 609 return nil, err 610 } 611 defer rows.Close() 612 613 for rows.Next() { 614 var status int 615 var count int64 616 err = rows.Scan(&status, &count) 617 if err != nil { 618 return nil, err 619 } 620 res[pb.ValidateErrorState(status)] = count 621 } 622 623 if err = rows.Err(); err != nil { 624 return nil, err 625 } 626 c.L.Info("error count loaded", zap.Reflect("counts", res)) 627 return res, nil 628 } 629 630 func (c *validatorPersistHelper) setRevision(rev int64) { 631 c.revision = rev 632 } 633 634 func (c *validatorPersistHelper) loadError(tctx *tcontext.Context, db *conn.BaseDB, filterState pb.ValidateErrorState) ([]*pb.ValidationError, error) { 635 var ( 636 rows *sql.Rows 637 err error 638 ) 639 res := make([]*pb.ValidationError, 0) 640 args := []interface{}{ 641 c.cfg.SourceID, 642 } 643 query := "SELECT id, source, src_schema_name, src_table_name, dst_schema_name, dst_table_name, data, dst_data, error_type, status, update_time " + 644 "FROM " + c.errorChangeTableName + " WHERE source = ?" 645 if filterState != pb.ValidateErrorState_InvalidErr { 646 query += " AND status=?" 647 args = append(args, int(filterState)) 648 } 649 // we do not retry, let user do it 650 rows, err = db.QueryContext(tctx, query, args...) 651 if err != nil { 652 return res, err 653 } 654 defer rows.Close() 655 for rows.Next() { 656 var ( 657 id, status, errType int 658 source, srcSchemaName, srcTableName, dstSchemaName, dstTableName, data, dstData, ts string 659 ) 660 err = rows.Scan(&id, &source, &srcSchemaName, &srcTableName, &dstSchemaName, &dstTableName, &data, &dstData, &errType, &status, &ts) 661 if err != nil { 662 return []*pb.ValidationError{}, err 663 } 664 res = append(res, &pb.ValidationError{ 665 Id: strconv.Itoa(id), 666 Source: source, 667 SrcTable: dbutil.TableName(srcSchemaName, srcTableName), 668 DstTable: dbutil.TableName(dstSchemaName, dstTableName), 669 SrcData: data, 670 DstData: dstData, 671 ErrorType: mapErrType2Str[validateFailedType(errType)], 672 Status: pb.ValidateErrorState(status), 673 Time: ts, 674 }) 675 } 676 if err = rows.Err(); err != nil { 677 return []*pb.ValidationError{}, err 678 } 679 c.L.Info("load validator errors", zap.Int("count", len(res))) 680 return res, nil 681 } 682 683 func (c *validatorPersistHelper) operateError(tctx *tcontext.Context, db *conn.BaseDB, validateOp pb.ValidationErrOp, errID uint64, isAll bool) error { 684 if validateOp == pb.ValidationErrOp_ClearErrOp { 685 return c.deleteError(tctx, db, errID, isAll) 686 } 687 query := "UPDATE " + c.errorChangeTableName + " SET status=? WHERE source=?" 688 var setStatus pb.ValidateErrorState 689 switch validateOp { 690 case pb.ValidationErrOp_IgnoreErrOp: 691 setStatus = pb.ValidateErrorState_IgnoredErr 692 case pb.ValidationErrOp_ResolveErrOp: 693 setStatus = pb.ValidateErrorState_ResolvedErr 694 default: 695 // unsupported op should be caught by caller 696 c.L.Warn("unsupported validator error operation", zap.Reflect("op", validateOp)) 697 return nil 698 } 699 args := []interface{}{ 700 int(setStatus), 701 c.cfg.SourceID, 702 } 703 if !isAll { 704 args = append(args, errID) 705 query += " AND id=?" 706 } 707 // we do not retry, let user do it 708 _, err := db.ExecContext(tctx, query, args...) 709 return err 710 } 711 712 func (c *validatorPersistHelper) deleteError(tctx *tcontext.Context, db *conn.BaseDB, errID uint64, isAll bool) error { 713 args := []interface{}{ 714 c.cfg.SourceID, 715 } 716 query := "DELETE FROM " + c.errorChangeTableName + " WHERE source=?" 717 if !isAll { 718 query += " AND id=?" 719 args = append(args, errID) 720 } 721 // we do not retry, let user do it 722 _, err := db.ExecContext(tctx, query, args...) 723 return err 724 }