github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/validate_worker.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 "context" 18 "database/sql" 19 "fmt" 20 "math" 21 "strconv" 22 "strings" 23 "sync" 24 "time" 25 26 "github.com/docker/go-units" 27 "github.com/pingcap/errors" 28 "github.com/pingcap/failpoint" 29 "github.com/pingcap/tidb/pkg/parser/model" 30 tidbmysql "github.com/pingcap/tidb/pkg/parser/mysql" 31 "github.com/pingcap/tidb/pkg/util/dbutil" 32 "github.com/pingcap/tidb/pkg/util/filter" 33 cdcmodel "github.com/pingcap/tiflow/cdc/model" 34 "github.com/pingcap/tiflow/dm/config" 35 "github.com/pingcap/tiflow/dm/pkg/conn" 36 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 37 "github.com/pingcap/tiflow/dm/pkg/log" 38 "github.com/pingcap/tiflow/dm/pkg/terror" 39 "github.com/pingcap/tiflow/dm/pkg/utils" 40 "github.com/pingcap/tiflow/dm/unit" 41 "github.com/pingcap/tiflow/pkg/sqlmodel" 42 "go.uber.org/atomic" 43 "go.uber.org/zap" 44 ) 45 46 const ( 47 workerChannelSize = 1000 48 49 maxAccumulatedRow = config.ValidatorMaxAccumulatedRow 50 queryTimeout = time.Minute 51 ) 52 53 type validateFailedType int 54 55 const ( 56 deletedRowExists validateFailedType = iota 57 rowNotExist 58 rowDifferent 59 ) 60 61 type validateFailedRow struct { 62 tp validateFailedType 63 dstData []*sql.NullString 64 65 srcJob *rowValidationJob 66 } 67 68 type validateWorker struct { 69 sync.Mutex 70 cfg config.ValidatorConfig 71 ctx context.Context 72 interval time.Duration 73 validator *DataValidator 74 L log.Logger 75 db *conn.BaseDB 76 rowChangeCh chan *rowValidationJob 77 batchSize int 78 rowErrorDelayInSec int64 79 maxPendingRowSize int64 80 maxPendingRowCount int64 81 82 pendingChangesMap map[string]*tableChangeJob 83 pendingRowCounts []int64 84 pendingRowSize int64 85 accuRowCount atomic.Int64 // accumulated row count from channel 86 errorRows []*validateFailedRow 87 } 88 89 func newValidateWorker(v *DataValidator, id int) *validateWorker { 90 workerLog := v.L.WithFields(zap.Int("id", id)) 91 rowErrorDelayInSec := int64(v.cfg.ValidatorCfg.RowErrorDelay.Duration.Seconds()) 92 // skip error, already check it in cfg.Adjust 93 maxPendingRowSize, _ := units.RAMInBytes(v.cfg.ValidatorCfg.MaxPendingRowSize) 94 return &validateWorker{ 95 cfg: v.cfg.ValidatorCfg, 96 ctx: v.ctx, 97 interval: v.validateInterval, 98 validator: v, 99 L: workerLog, 100 db: v.toDB, 101 rowChangeCh: make(chan *rowValidationJob, workerChannelSize), 102 batchSize: v.cfg.ValidatorCfg.BatchQuerySize, 103 rowErrorDelayInSec: rowErrorDelayInSec, 104 maxPendingRowSize: maxPendingRowSize, 105 maxPendingRowCount: int64(v.cfg.ValidatorCfg.MaxPendingRowCount), 106 107 pendingChangesMap: make(map[string]*tableChangeJob), 108 pendingRowCounts: make([]int64, rowChangeTypeCount), 109 } 110 } 111 112 func (vw *validateWorker) run() { 113 validatedBeforeTimer := false 114 outer: 115 for { 116 select { 117 case change, ok := <-vw.rowChangeCh: 118 if !ok { 119 break outer 120 } 121 if change.Tp == flushCheckpoint { 122 // validate before flush to reduce the number of row changes 123 vw.validateTableChange() 124 validatedBeforeTimer = true 125 change.wg.Done() 126 break 127 } 128 129 vw.updateRowChange(change) 130 vw.accuRowCount.Add(1) 131 // reduce number of pending rows 132 if vw.accuRowCount.Load() >= maxAccumulatedRow { 133 vw.validateTableChange() 134 validatedBeforeTimer = true 135 } 136 case <-time.After(vw.interval): 137 // reduce the number of validation 138 if validatedBeforeTimer { 139 validatedBeforeTimer = false 140 break 141 } 142 vw.validateTableChange() 143 } 144 } 145 } 146 147 func (vw *validateWorker) updateRowChange(job *rowValidationJob) { 148 vw.Lock() 149 defer vw.Unlock() 150 // cluster using target table 151 tbl := job.row.GetTargetTable() 152 targetTable := filter.Table{Schema: tbl.Schema, Name: tbl.Table} 153 fullTableName := targetTable.String() 154 change := vw.pendingChangesMap[fullTableName] 155 if change == nil { 156 // no change of this table 157 change = newTableChangeJob() 158 vw.pendingChangesMap[fullTableName] = change 159 } 160 if change.addOrUpdate(job) { 161 vw.newJobAdded(job) 162 } 163 } 164 165 func (vw *validateWorker) validateTableChange() { 166 var err error 167 defer func() { 168 if err != nil && !isRetryableValidateError(err) { 169 vw.validator.sendError(terror.ErrValidatorValidateChange.Delegate(err)) 170 } 171 if panicErr := recover(); panicErr != nil { 172 vw.L.Error("worker panic", zap.Any("err", panicErr)) 173 vw.validator.sendError(terror.ErrValidatorPanic.Generate(panicErr)) 174 } 175 }() 176 177 // clear accumulated row counter 178 vw.accuRowCount.Store(0) 179 180 if vw.getAllPendingRowCount() == 0 { 181 vw.L.Debug("pending row count = 0, skip validation") 182 return 183 } 184 185 failedChanges := make(map[string]map[string]*validateFailedRow) 186 for k, tblChange := range vw.pendingChangesMap { 187 var insertUpdateChanges, deleteChanges []*rowValidationJob 188 for _, r := range tblChange.jobs { 189 if r.Tp == rowDeleted { 190 deleteChanges = append(deleteChanges, r) 191 } else { 192 insertUpdateChanges = append(insertUpdateChanges, r) 193 } 194 } 195 allFailedRows := make(map[string]*validateFailedRow) 196 validateFunc := func(rows []*rowValidationJob, isDelete bool) error { 197 if len(rows) == 0 { 198 return nil 199 } 200 failedRows, err2 := vw.validateRowChanges(rows, isDelete) 201 if err2 != nil { 202 return err2 203 } 204 for key, val := range failedRows { 205 allFailedRows[key] = val 206 } 207 return nil 208 } 209 if err = validateFunc(insertUpdateChanges, false); err != nil { 210 return 211 } 212 if err = validateFunc(deleteChanges, true); err != nil { 213 return 214 } 215 if len(allFailedRows) > 0 { 216 failedChanges[k] = allFailedRows 217 } 218 } 219 220 vw.updatePendingAndErrorRows(failedChanges) 221 222 // check whether we need to stop validation 223 pendingRowSize := vw.validator.getPendingRowSize() 224 allPendingRowCount := vw.validator.getAllPendingRowCount() 225 if pendingRowSize > vw.maxPendingRowSize || allPendingRowCount > vw.maxPendingRowCount { 226 vw.validator.sendError(terror.ErrValidatorTooMuchPending.Generate( 227 pendingRowSize, vw.maxPendingRowSize, 228 allPendingRowCount, vw.maxPendingRowCount), 229 ) 230 } 231 } 232 233 func (vw *validateWorker) updatePendingAndErrorRows(failedChanges map[string]map[string]*validateFailedRow) { 234 vw.Lock() 235 defer vw.Unlock() 236 237 newPendingCnt := make([]int64, rowChangeTypeCount) 238 newPendingRowSize := int64(0) 239 allErrorRows := make([]*validateFailedRow, 0) 240 newPendingChanges := make(map[string]*tableChangeJob) 241 validateTS := time.Now().Unix() 242 for tblKey, rows := range failedChanges { 243 tblChange := vw.pendingChangesMap[tblKey] 244 newPendingRows := make(map[string]*rowValidationJob) 245 for pk, row := range rows { 246 job := tblChange.jobs[pk] 247 if vw.validator.isMarkErrorStarted() { 248 job.FailedCnt++ 249 if job.FirstValidateTS == 0 { 250 job.FirstValidateTS = validateTS 251 } 252 253 if validateTS-job.FirstValidateTS >= vw.rowErrorDelayInSec { 254 row.srcJob = job 255 allErrorRows = append(allErrorRows, row) 256 } else { 257 newPendingRows[pk] = job 258 newPendingCnt[job.Tp]++ 259 newPendingRowSize += int64(job.size) 260 } 261 } else { 262 newPendingRows[pk] = job 263 newPendingCnt[job.Tp]++ 264 newPendingRowSize += int64(job.size) 265 } 266 } 267 if len(newPendingRows) > 0 { 268 newPendingChanges[tblKey] = &tableChangeJob{ 269 jobs: newPendingRows, 270 } 271 } 272 } 273 274 vw.L.Debug("pending row count (insert, update, delete)", zap.Int64s("before", vw.pendingRowCounts), 275 zap.Int64s("after", newPendingCnt)) 276 vw.setPendingRowCountsAndSize(newPendingCnt, newPendingRowSize) 277 vw.pendingChangesMap = newPendingChanges 278 vw.errorRows = append(vw.errorRows, allErrorRows...) 279 vw.validator.incrErrorRowCount(len(allErrorRows)) 280 } 281 282 func (vw *validateWorker) validateRowChanges(rows []*rowValidationJob, deleteChange bool) (map[string]*validateFailedRow, error) { 283 res := make(map[string]*validateFailedRow) 284 for start := 0; start < len(rows); start += vw.batchSize { 285 end := start + vw.batchSize 286 if end > len(rows) { 287 end = len(rows) 288 } 289 batch := rows[start:end] 290 failedRows, err := vw.batchValidateRowChanges(batch, deleteChange) 291 if err != nil { 292 return nil, err 293 } 294 for k, v := range failedRows { 295 res[k] = v 296 } 297 } 298 return res, nil 299 } 300 301 func (vw *validateWorker) getPendingChangesMap() map[string]*tableChangeJob { 302 vw.Lock() 303 defer vw.Unlock() 304 return vw.pendingChangesMap 305 } 306 307 func (vw *validateWorker) getErrorRows() []*validateFailedRow { 308 vw.Lock() 309 defer vw.Unlock() 310 return vw.errorRows 311 } 312 313 func (vw *validateWorker) batchValidateRowChanges(rows []*rowValidationJob, deleteChange bool) (map[string]*validateFailedRow, error) { 314 failpoint.Inject("ValidatorWorkerPanic", func() {}) 315 316 pkValues := make([][]string, 0, len(rows)) 317 for _, r := range rows { 318 pkValues = append(pkValues, r.row.RowStrIdentity()) 319 } 320 firstRow := rows[0].row 321 cond := &Cond{ 322 TargetTbl: firstRow.TargetTableID(), 323 Columns: firstRow.SourceTableInfo().Columns, 324 PK: firstRow.UniqueNotNullIdx(), 325 PkValues: pkValues, 326 } 327 var failedRows map[string]*validateFailedRow 328 var err error 329 if deleteChange { 330 failedRows, err = vw.validateDeletedRows(cond) 331 } else { 332 failedRows, err = vw.validateInsertAndUpdateRows(rows, cond) 333 } 334 if err != nil { 335 return nil, err 336 } 337 return failedRows, nil 338 } 339 340 func (vw *validateWorker) validateDeletedRows(cond *Cond) (map[string]*validateFailedRow, error) { 341 targetRows, err := vw.getTargetRows(cond) 342 if err != nil { 343 return nil, err 344 } 345 346 failedRows := make(map[string]*validateFailedRow, len(targetRows)) 347 for key, val := range targetRows { 348 failedRows[key] = &validateFailedRow{tp: deletedRowExists, dstData: val} 349 } 350 return failedRows, nil 351 } 352 353 func (vw *validateWorker) validateInsertAndUpdateRows(rows []*rowValidationJob, cond *Cond) (map[string]*validateFailedRow, error) { 354 failedRows := make(map[string]*validateFailedRow) 355 sourceRows := getSourceRowsForCompare(rows) 356 targetRows, err := vw.getTargetRows(cond) 357 if err != nil { 358 return nil, err 359 } 360 361 if len(targetRows) > len(sourceRows) { 362 // if this happens, downstream should have removed the primary key 363 vw.L.Debug("more data on downstream, may come from other client") 364 } 365 366 firstRow := rows[0].row 367 tableInfo := firstRow.SourceTableInfo() 368 validateContext := &validateCompareContext{ 369 logger: vw.L, 370 sourceTable: firstRow.GetSourceTable(), 371 targetTable: firstRow.GetTargetTable(), 372 columns: tableInfo.Columns, 373 } 374 for key, sourceRow := range sourceRows { 375 targetRow, ok := targetRows[key] 376 if !ok { 377 failedRows[key] = &validateFailedRow{tp: rowNotExist} 378 continue 379 } 380 if vw.cfg.Mode == config.ValidationFull { 381 // only compare the whole row in full mode 382 eq, err2 := validateContext.compareData(key, sourceRow, targetRow) 383 if err2 != nil { 384 return nil, err2 385 } 386 if !eq { 387 failedRows[key] = &validateFailedRow{tp: rowDifferent, dstData: targetRow} 388 } 389 } 390 } 391 return failedRows, nil 392 } 393 394 func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*sql.NullString, error) { 395 ctx, cancelFunc := context.WithTimeout(vw.ctx, queryTimeout) 396 defer cancelFunc() 397 tctx := tcontext.NewContext(ctx, vw.L) 398 columnNames := make([]string, 0, len(cond.Columns)) 399 for _, col := range cond.Columns { 400 columnNames = append(columnNames, dbutil.ColumnName(col.Name.O)) 401 } 402 columns := strings.Join(columnNames, ", ") 403 rowsQuery := fmt.Sprintf("SELECT /*!40001 SQL_NO_CACHE */ %s FROM %s WHERE %s", 404 columns, cond.TargetTbl, cond.GetWhere()) 405 // query using sql.DB directly, BaseConn is more than what we need 406 rows, err := vw.db.QueryContext(tctx, rowsQuery, cond.GetArgs()...) 407 if err != nil { 408 if isRetryableValidateError(err) { 409 vw.L.Info("met retryable error", zap.Error(err)) 410 } else { 411 vw.L.Error("failed to query", 412 zap.String("query", utils.TruncateString(rowsQuery, -1)), 413 zap.String("args", utils.TruncateInterface(cond.GetArgs(), -1))) 414 err = errors.Trace(err) 415 } 416 return nil, err 417 } 418 defer rows.Close() 419 420 result := make(map[string][]*sql.NullString) 421 for rows.Next() { 422 rowData, err := scanRow(rows) 423 if err != nil { 424 return nil, err 425 } 426 pkCols := cond.PK.Columns 427 pkValues := make([]string, 0, len(pkCols)) 428 for _, col := range pkCols { 429 pkValues = append(pkValues, rowData[col.Offset].String) 430 } 431 pk := genRowKeyByString(pkValues) 432 result[pk] = rowData 433 } 434 return result, rows.Err() 435 } 436 437 func (vw *validateWorker) close() { 438 close(vw.rowChangeCh) 439 } 440 441 func (vw *validateWorker) resetErrorRows() { 442 vw.Lock() 443 defer vw.Unlock() 444 vw.errorRows = make([]*validateFailedRow, 0) 445 } 446 447 func (vw *validateWorker) newJobAdded(job *rowValidationJob) { 448 tp := job.Tp 449 vw.pendingRowCounts[tp]++ 450 vw.validator.addPendingRowCount(tp, 1) 451 452 vw.pendingRowSize += int64(job.size) 453 vw.validator.addPendingRowSize(int64(job.size)) 454 } 455 456 func (vw *validateWorker) setPendingRowCountsAndSize(newCounts []int64, newSize int64) { 457 for tp, val := range newCounts { 458 diff := val - vw.pendingRowCounts[tp] 459 vw.pendingRowCounts[tp] = val 460 vw.validator.addPendingRowCount(rowChangeJobType(tp), diff) 461 } 462 463 diff := newSize - vw.pendingRowSize 464 vw.pendingRowSize = newSize 465 vw.validator.addPendingRowSize(diff) 466 } 467 468 func (vw *validateWorker) getAllPendingRowCount() int64 { 469 return vw.pendingRowCounts[rowInsert] + vw.pendingRowCounts[rowUpdated] + vw.pendingRowCounts[rowDeleted] 470 } 471 472 type validateCompareContext struct { 473 logger log.Logger 474 sourceTable *cdcmodel.TableName 475 targetTable *cdcmodel.TableName 476 columns []*model.ColumnInfo 477 } 478 479 // a simplified version of https://github.com/pingcap/tidb-tools/blob/d9fdfa2f9040aab3fab7cd11774a82226f467fe7/sync_diff_inspector/utils/utils.go#L487-L606 480 func (c *validateCompareContext) compareData(key string, sourceData, targetData []*sql.NullString) (bool, error) { 481 for i, column := range c.columns { 482 data1, data2 := sourceData[i], targetData[i] 483 if data1.Valid != data2.Valid { 484 return false, nil 485 } 486 str1, str2 := data1.String, data2.String 487 if str1 == str2 { 488 continue 489 } else if column.FieldType.GetType() == tidbmysql.TypeFloat || column.FieldType.GetType() == tidbmysql.TypeDouble { 490 // source and target data have different precision? 491 num1, err1 := strconv.ParseFloat(str1, 64) 492 num2, err2 := strconv.ParseFloat(str2, 64) 493 if err1 != nil || err2 != nil { 494 // should not happen 495 return false, errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", str1, str2, err1, err2) 496 } 497 if math.Abs(num1-num2) <= 1e-6 { 498 continue 499 } 500 } 501 if c.logger.Core().Enabled(zap.DebugLevel) { 502 c.logger.Debug("compare failed", 503 zap.Stringer("src table", c.sourceTable), 504 zap.Stringer("dst table", c.targetTable), 505 zap.String("col", fmt.Sprintf("%s %s", column.Name, column.GetTypeDesc())), 506 zap.String("key", key), 507 zap.Reflect("src data", data1), zap.Reflect("dst data", data2)) 508 } 509 return false, nil 510 } 511 512 return true, nil 513 } 514 515 func isRetryableValidateError(err error) bool { 516 err = errors.Cause(err) 517 return err == context.DeadlineExceeded || isRetryableDBError(err) 518 } 519 520 func isRetryableDBError(err error) bool { 521 return unit.IsResumableDBError(err) 522 } 523 524 func scanRow(rows *sql.Rows) ([]*sql.NullString, error) { 525 cols, err := rows.Columns() 526 if err != nil { 527 return nil, errors.Trace(err) 528 } 529 530 colVals := make([][]byte, len(cols)) 531 colValsI := make([]interface{}, len(colVals)) 532 for i := range colValsI { 533 colValsI[i] = &colVals[i] 534 } 535 536 err = rows.Scan(colValsI...) 537 if err != nil { 538 return nil, errors.Trace(err) 539 } 540 541 result := make([]*sql.NullString, len(cols)) 542 for i := range colVals { 543 result[i] = &sql.NullString{ 544 String: string(colVals[i]), 545 Valid: colVals[i] != nil, 546 } 547 } 548 549 return result, nil 550 } 551 552 func getSourceRowsForCompare(jobs []*rowValidationJob) map[string][]*sql.NullString { 553 rowMap := make(map[string][]*sql.NullString, len(jobs)) 554 for _, j := range jobs { 555 r := j.row 556 colValues := make([]*sql.NullString, r.ColumnCount()) 557 rowValues := r.RowValues() 558 for i := range rowValues { 559 var colData string 560 if rowValues[i] != nil { 561 colData = sqlmodel.ColValAsStr(rowValues[i]) 562 } 563 colValues[i] = &sql.NullString{ 564 String: colData, 565 Valid: rowValues[i] != nil, 566 } 567 } 568 rowMap[j.Key] = colValues 569 } 570 return rowMap 571 }