From f37a133bc8e629ff9483e3bc97bd38018f36759d Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Wed, 9 Feb 2022 14:44:07 +0800 Subject: [PATCH 01/36] feat: implement validator --- dm/syncer/data_validator.go | 822 +++++++++++++++++++++++++++++++++++- dm/syncer/validator_cond.go | 59 +++ 2 files changed, 879 insertions(+), 2 deletions(-) create mode 100644 dm/syncer/validator_cond.go diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 1eb368e9bac..e394478996a 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -15,12 +15,27 @@ package syncer import ( "context" + "database/sql" + "hash/fnv" + "sort" + + "github.com/pingcap/errors" + + "fmt" + "math" + "strconv" + "strings" "sync" "time" + "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" + tidbmysql "github.com/pingcap/tidb/parser/mysql" + "go.uber.org/atomic" "go.uber.org/zap" + "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/tiflow/dm/dm/unit" @@ -32,6 +47,96 @@ import ( "github.com/pingcap/tiflow/dm/syncer/dbconn" ) +const ( + defaultDelay = 5 * time.Second + retryInterval = 5 * time.Second + validationInterval = time.Second + batchRowCount = 200 + defaultWorkerCnt = 5 +) + +type rowChangeType int + +// table info +type TableDiff struct { + // Schema represents the database name. + Schema string `json:"schema"` + + // Table represents the table name. + Table string `json:"table"` + + // Info is the parser.TableInfo, include some meta infos for this table. + // It used for TiDB/MySQL/MySQL Shard sources. + Info *model.TableInfo `json:"info"` + + PrimaryKey *model.IndexInfo + ColumnMap map[string]*model.ColumnInfo +} + +type RowDataIterator interface { + // Next seeks the next row data, it used when compared rows. + Next() (map[string]*dbutil.ColumnData, error) + // Close release the resource. + Close() +} + +type RowDataIteratorImpl struct { + Rows *sql.Rows +} + +func (r *RowDataIteratorImpl) Next() (map[string]*dbutil.ColumnData, error) { + for r.Rows.Next() { + rowData, err := dbutil.ScanRow(r.Rows) + return rowData, err + } + return nil, nil +} + +func (r *RowDataIteratorImpl) Close() { + r.Rows.Close() +} + +type RowChangeIteratorImpl struct { + Rows []map[string]*dbutil.ColumnData + Idx int +} + +func (b *RowChangeIteratorImpl) Next() (map[string]*dbutil.ColumnData, error) { + if b.Idx >= len(b.Rows) { + return nil, nil + } + row := b.Rows[b.Idx] + b.Idx++ + return row, nil +} + +func (b *RowChangeIteratorImpl) Close() { + // skip: nothing to do +} + +const ( + rowInvalidChange rowChangeType = iota + rowInsert + rowDeleted + rowUpdated +) + +// change of table +// binlog changes are clustered into table changes +// the validator validates changes of table-grain at a time +type tableChange struct { + table *TableDiff + rows map[string]*rowChange +} + +// change of a row +type rowChange struct { + pk []string + data []interface{} + theType rowChangeType + lastMeetTs int64 // the last meet timestamp(in seconds) +} + // DataValidator // validator can be start when there's syncer unit in the subtask and validation mode is not none, // it's terminated when the subtask is terminated. @@ -52,11 +157,27 @@ type DataValidator struct { L log.Logger fromDB *conn.BaseDB + fromDBConn *dbconn.DBConn + toDB *conn.BaseDB + toDBConn *dbconn.DBConn timezone *time.Location syncCfg replication.BinlogSyncerConfig streamerController *StreamerController result pb.ProcessResult + + failedRowCnt atomic.Int64 + accumulatedChanges map[string]*tableChange + pendingRowCnt atomic.Int64 + rowsEventChan chan *replication.BinlogEvent // unbuffered is enough + pendingChangeCh chan map[string]*tableChange + changeEventCount []int + validationTimer *time.Timer + diffTables map[string]*TableDiff + workerCnt int + pendingChangeChs map[int]chan map[string]*tableChange // replace pendingChangeCh + failedChangesMap map[int]map[string]*tableChange // replace failedChanges + workerLocks map[int]*sync.RWMutex // lock for each worker } func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *DataValidator { @@ -83,11 +204,18 @@ func (v *DataValidator) initialize() error { dbCfg := v.cfg.From dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxDMLConnectionTimeout) - v.fromDB, err = dbconn.CreateBaseDB(&dbCfg) + // v.fromDB, err = dbconn.CreateBaseDB(&dbCfg) + var fromDBConns, toDBConns []*dbconn.DBConn + v.fromDB, fromDBConns, err = dbconn.CreateConns(tctx, v.cfg, &dbCfg, 1) if err != nil { return err } - + v.fromDBConn = fromDBConns[0] + v.toDB, toDBConns, err = dbconn.CreateConns(tctx, v.cfg, &dbCfg, 1) + if err != nil { + return err + } + v.toDBConn = toDBConns[0] v.timezone, err = str2TimezoneOrFromDB(tctx, v.cfg.Timezone, &v.cfg.To) if err != nil { return err @@ -100,6 +228,12 @@ func (v *DataValidator) initialize() error { v.streamerController = NewStreamerController(v.syncCfg, v.cfg.EnableGTID, &dbconn.UpStreamConn{BaseDB: v.fromDB}, v.cfg.RelayDir, v.timezone, nil) + v.workerCnt = defaultWorkerCnt + for i := 0; i < v.workerCnt; i++ { + v.pendingChangeChs[i] = make(chan map[string]*tableChange) + v.failedChangesMap[i] = make(map[string]*tableChange) + v.workerLocks[i] = &sync.RWMutex{} + } return nil } @@ -167,6 +301,41 @@ func (v *DataValidator) doValidate() { } v.L.Info("start continuous validation") + v.wg.Add(2) + go v.rowsEventProcessRoutine() + go v.validateTaskDispatchRoutine() + v.wg.Add(v.workerCnt) + for i := 0; i < v.workerCnt; i++ { + go v.retryFailedRowsByWorker(i) + } + var latestPos mysql.Position + for { + e, err := v.streamerController.GetEvent(tctx) + if err != nil { + v.L.Fatal("getting event from streamer controller failed") + return + } + // todo: configuring the time or use checkpoint + eventTime := time.Unix(int64(e.Header.Timestamp), 0) + lag := time.Since(eventTime) + if lag < defaultDelay { + time.Sleep(defaultDelay - lag) + } + + switch ev := e.Event.(type) { + case *replication.RotateEvent: + latestPos.Name = string(ev.NextLogName) + case *replication.QueryEvent: + // TODO not processed now + case *replication.RowsEvent: + select { + case v.rowsEventChan <- e: + case <-v.ctx.Done(): + return + } + } + latestPos.Pos = e.Header.LogPos + } } func (v *DataValidator) Stop() { @@ -179,6 +348,7 @@ func (v *DataValidator) Stop() { v.streamerController.Close() v.fromDB.Close() + v.toDB.Close() if v.cancel != nil { v.cancel() @@ -198,3 +368,651 @@ func (v *DataValidator) Stage() pb.Stage { defer v.RUnlock() return v.stage } + +func (v *DataValidator) retryFailedRowsByWorker(workerID int) { + v.wg.Done() + for { + v.workerLocks[workerID].Lock() + retryChange := v.failedChangesMap[workerID] + failed := v.validateTableChange(retryChange) + deltaFailed := v.updateFailedChangesByWorker(retryChange, failed, workerID) + v.failedRowCnt.Add(int64(deltaFailed)) + v.workerLocks[workerID].Unlock() + // todo: print validation summary + select { + case <-v.ctx.Done(): + return + case <-time.After(retryInterval): + } + } +} + +func (v *DataValidator) rowsEventProcessRoutine() { + v.wg.Done() + for { + select { + case <-v.ctx.Done(): + return + case e := <-v.rowsEventChan: + if err := v.processEventRows(e.Header, e.Event.(*replication.RowsEvent)); err != nil { + v.L.Warn("failed to process event: ", zap.Reflect("error", err)) + } + case <-v.validationTimer.C: + rowCount := v.getRowCount(v.accumulatedChanges) + if rowCount > 0 { + v.pendingChangeCh <- v.accumulatedChanges + v.accumulatedChanges = make(map[string]*tableChange) + } + v.validationTimer.Reset(validationInterval) + } + } +} + +func (v *DataValidator) workerValidateTableChange(workerID int) { + v.wg.Done() + for { + select { + case change := <-v.pendingChangeChs[workerID]: + v.workerLocks[workerID].Lock() + // 1. validate table change + // 2. update failed rows + // 3. update pending row count + // 4. update failed row cnt + failed := v.validateTableChange(change) + deltaFailed := v.updateFailedChangesByWorker(change, failed, workerID) + v.pendingRowCnt.Sub(int64(v.getRowCount(change))) // atomic value + v.failedRowCnt.Add(int64(deltaFailed)) // atomic value + v.workerLocks[workerID].Unlock() + case <-v.ctx.Done(): + return + } + + } +} + +func (v *DataValidator) validateTaskDispatchRoutine() { + v.wg.Done() + // start workers here + v.wg.Add(v.workerCnt) + for i := 0; i < v.workerCnt; i++ { + go v.workerValidateTableChange(i) + } + for { + select { + case change := <-v.pendingChangeCh: + v.dispatchRowChange(change) // dispatch change to worker + case <-v.ctx.Done(): + return + } + } +} + +func hashTablePk(s string) uint32 { + h := fnv.New32a() + h.Write([]byte(s)) + return h.Sum32() +} + +func (v *DataValidator) dispatchRowChange(change map[string]*tableChange) { + dispatchMap := make(map[int]map[string]*tableChange, 0) + for tableName := range change { + // for every table + for _, curRowChange := range change[tableName].rows { + // for every row in the table + // 1. join primary key by '-' + // 2. hash (tableName, primaryKey) to hashVal + // 3. dispatch the row change to dispatchMap[hashVal][tableName] + pk := strings.Join(curRowChange.pk, "-") + hashKey := tableName + "," + pk + hashVal := int(hashTablePk(hashKey)) % v.workerCnt + if _, ok := dispatchMap[hashVal]; !ok { + dispatchMap[hashVal] = make(map[string]*tableChange, 0) + } + if _, ok := dispatchMap[hashVal][tableName]; !ok { + dispatchMap[hashVal][tableName] = &tableChange{ + table: change[tableName].table, + rows: make(map[string]*rowChange, 0), + } + } + dispatchMap[hashVal][tableName].rows[pk] = curRowChange + } + } + for hashVal := range dispatchMap { + v.pendingChangeChs[hashVal] <- dispatchMap[hashVal] + } +} + +func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *replication.RowsEvent) error { + schemaName, tableName := string(ev.Table.Schema), string(ev.Table.Table) + fullTableName := fmt.Sprintf("%s.%s", schemaName, tableName) + var table *TableDiff + if _, ok := v.diffTables[fullTableName]; !ok { + // table not found in cache + // try getting table info from the upstream + tctx := tcontext.NewContext(v.ctx, v.L) + fullTableQueryName := fmt.Sprintf("`%s`.`%s`", schemaName, tableName) + createSQL, err := dbconn.GetTableCreateSQL(tctx, v.fromDBConn, fullTableQueryName) + if err != nil { + // get create table stmt failed + return err + } + parser, err := utils.GetParser(v.ctx, v.fromDB.DB) + if err != nil { + return err + } + tableInfo, err := dbutil.GetTableInfoBySQL(createSQL, parser) + if err != nil { + return err + } + columnMap := make(map[string]*model.ColumnInfo) + for _, col := range tableInfo.Columns { + columnMap[col.Name.O] = col + } + var primaryIdx *model.IndexInfo + for _, idx := range tableInfo.Indices { + if idx.Primary { + primaryIdx = idx + } + } + v.diffTables[fullTableName] = &TableDiff{ + Schema: schemaName, + Table: tableName, + Info: tableInfo, + PrimaryKey: primaryIdx, + ColumnMap: columnMap, + } + table = v.diffTables[fullTableName] + } + if table == nil { + return nil + } + if table.PrimaryKey == nil { + errMsg := fmt.Sprintf("no primary key for %s.%s", table.Schema, table.Table) + return errors.New(errMsg) + } + for _, cols := range ev.SkippedColumns { + if len(cols) > 0 { + return errors.New("") + } + } + changeType := getRowChangeType(header.EventType) + if changeType == rowInvalidChange { + v.L.Info("ignoring unrecognized event", zap.Reflect("event header", header)) + return nil + } + v.changeEventCount[changeType]++ + + init, step := 0, 1 + if changeType == rowUpdated { + init, step = 1, 2 + } + pk := table.PrimaryKey + pkIndices := make([]int, len(pk.Columns)) + for i, col := range pk.Columns { + pkIndices[i] = table.ColumnMap[col.Name.O].Offset + } + + rowCount := v.getRowCount(v.accumulatedChanges) + change := v.accumulatedChanges[fullTableName] + for i := init; i < len(ev.Rows); i += step { + row := ev.Rows[i] + pkValue := make([]string, len(pk.Columns)) + for _, idx := range pkIndices { + pkValue[idx] = fmt.Sprintf("%v", row[idx]) + } + + if change == nil { + // no change of this table + change = &tableChange{ + table: table, + rows: make(map[string]*rowChange), + } + v.accumulatedChanges[fullTableName] = change + } + key := strings.Join(pkValue, "-") + val, ok := change.rows[key] + if !ok { + // this row hasn't been changed before + val = &rowChange{pk: pkValue} + change.rows[key] = val + rowCount++ + v.pendingRowCnt.Inc() + } + val.data = row + val.theType = changeType + val.lastMeetTs = int64(header.Timestamp) + + if rowCount >= batchRowCount { + v.pendingChangeCh <- v.accumulatedChanges + v.accumulatedChanges = make(map[string]*tableChange) + + if !v.validationTimer.Stop() { + <-v.validationTimer.C + } + v.validationTimer.Reset(validationInterval) + + rowCount = 0 + change = nil + } + } + return nil +} + +func (v *DataValidator) getRowCount(c map[string]*tableChange) int { + res := 0 + for _, val := range c { + res += len(val.rows) + } + return res +} + +func getRowChangeType(t replication.EventType) rowChangeType { + switch t { + case replication.WRITE_ROWS_EVENTv0, replication.WRITE_ROWS_EVENTv1, replication.WRITE_ROWS_EVENTv2: + return rowInsert + case replication.UPDATE_ROWS_EVENTv0, replication.UPDATE_ROWS_EVENTv1, replication.UPDATE_ROWS_EVENTv2: + return rowUpdated + case replication.DELETE_ROWS_EVENTv0, replication.DELETE_ROWS_EVENTv1, replication.DELETE_ROWS_EVENTv2: + return rowDeleted + default: + return rowInvalidChange + } +} + +func (v *DataValidator) validateTableChange(tableChanges map[string]*tableChange) map[string]*tableChange { + failedChanges := make(map[string]*tableChange) + for k, val := range tableChanges { + var insertUpdateChanges, deleteChanges []*rowChange + for _, r := range val.rows { + if r.theType == rowDeleted { + deleteChanges = append(deleteChanges, r) + } else { + insertUpdateChanges = append(insertUpdateChanges, r) + } + } + rows := make(map[string]*rowChange, 0) + if len(insertUpdateChanges) > 0 { + failedRows := v.validateChanges(val.table, insertUpdateChanges, false) + for _, pk := range failedRows { + key := strings.Join(pk, "-") + rows[key] = val.rows[key] + } + } + if len(deleteChanges) > 0 { + failedRows := v.validateChanges(val.table, deleteChanges, true) + for _, pk := range failedRows { + key := strings.Join(pk, "-") + rows[key] = val.rows[key] + } + } + if len(rows) > 0 { + failedChanges[k] = &tableChange{ + table: val.table, + rows: rows, + } + } + } + return failedChanges +} + +func (v *DataValidator) validateChanges(table *TableDiff, rows []*rowChange, deleteChange bool) [][]string { + pkValues := make([][]string, 0, len(rows)) + for _, r := range rows { + pkValues = append(pkValues, r.pk) + } + cond := &Cond{Table: table, PkValues: pkValues} + var failedRows [][]string + var err error + if deleteChange { + failedRows, err = v.validateDeletedRows(cond) + } else { + failedRows, err = v.validateInsertAndUpdateRows(rows, cond) + } + if err != nil { + panic(err) + } + return failedRows +} + +// remove previous failed rows related to current batch of rows +// e.g. Assuming that one row was modified twice and successfully migrated to downstream. +// the validator might get false negative result when validating the first update binlog record +// but it must finally get true positive after validating the second update record. +// This function updates the failed rows every time after validating +func (v *DataValidator) updateFailedChangesByWorker(all, failed map[string]*tableChange, workerID int) int { + failedChanges := v.failedChangesMap[workerID] + deltaFailed := 0 + for k, val := range all { + // remove from all + prevFailed := failedChanges[k] + if prevFailed == nil { + continue + } + for _, r := range val.rows { + key := strings.Join(r.pk, "-") + delete(prevFailed.rows, key) + deltaFailed-- + } + } + for k, val := range failed { + // add from failed + prevFailed := failedChanges[k] + if prevFailed == nil { + prevFailed = &tableChange{ + table: val.table, + rows: make(map[string]*rowChange), + } + failedChanges[k] = prevFailed + } + + for _, r := range val.rows { + key := strings.Join(r.pk, "-") + prevFailed.rows[key] = r + deltaFailed++ + } + } + return deltaFailed +} + +func (v *DataValidator) validateDeletedRows(cond *Cond) ([][]string, error) { + downstreamRowsIterator, err := v.getRowsFrom(cond, v.toDBConn) + if err != nil { + return nil, errors.Trace(err) + } + defer downstreamRowsIterator.Close() + + var failedRows [][]string + for { + data, err := downstreamRowsIterator.Next() + if err != nil { + return nil, err + } + if data == nil { + break + } + failedRows = append(failedRows, getPkValues(data, cond)) + } + return failedRows, nil +} + +func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Cond) ([][]string, error) { + var failedRows [][]string + var upstreamRowsIterator, downstreamRowsIterator RowDataIterator + var err error + upstreamRowsIterator, err = v.getRowChangeIterator(cond.Table, rows) + if err != nil { + return nil, errors.New("") + } + defer upstreamRowsIterator.Close() + downstreamRowsIterator, err = v.getRowsFrom(cond, v.toDBConn) + if err != nil { + return nil, errors.New("") + } + defer downstreamRowsIterator.Close() + + var lastUpstreamData, lastDownstreamData map[string]*dbutil.ColumnData + + tableInfo := cond.Table.Info + _, orderKeyCols := dbutil.SelectUniqueOrderKey(tableInfo) + for { + if lastUpstreamData == nil { + lastUpstreamData, err = upstreamRowsIterator.Next() + if err != nil { + return nil, err + } + } + + if lastDownstreamData == nil { + lastDownstreamData, err = downstreamRowsIterator.Next() + if err != nil { + return nil, err + } + } + + // may have deleted on upstream and haven't synced to downstream, + // we mark this as success as we'll check the delete-event later + // or downstream removed the pk and added more data by other clients, skip it. + if lastUpstreamData == nil && lastDownstreamData != nil { + v.L.Debug("more data on downstream, may come from other client, skip it") + break + } + + if lastDownstreamData == nil { + // target lack some data, should insert the last source datas + for lastUpstreamData != nil { + failedRows = append(failedRows, getPkValues(lastUpstreamData, cond)) + + lastUpstreamData, err = upstreamRowsIterator.Next() + if err != nil { + return nil, err + } + } + break + } + + eq, cmp, err := v.CompareData(lastUpstreamData, lastDownstreamData, orderKeyCols, tableInfo.Columns) + if err != nil { + return nil, errors.New("") + } + if eq { + lastDownstreamData = nil + lastUpstreamData = nil + continue + } + + switch cmp { + case 1: + // may have deleted on upstream and haven't synced to downstream, + // we mark this as success as we'll check the delete-event later + // or downstream removed the pk and added more data by other clients, skip it. + v.L.Debug("more data on downstream, may come from other client, skip it", zap.Reflect("data", lastDownstreamData)) + lastDownstreamData = nil + case -1: + failedRows = append(failedRows, getPkValues(lastUpstreamData, cond)) + lastUpstreamData = nil + case 0: + failedRows = append(failedRows, getPkValues(lastUpstreamData, cond)) + lastUpstreamData = nil + lastDownstreamData = nil + } + } + return failedRows, nil +} + +func (v *DataValidator) getRowsFrom(cond *Cond, conn *dbconn.DBConn) (RowDataIterator, error) { + tctx := tcontext.NewContext(v.ctx, v.L) + fullTableName := dbutil.TableName(cond.Table.Schema, cond.Table.Table) + orderKeys, _ := dbutil.SelectUniqueOrderKey(cond.Table.Info) + columnNames := make([]string, 0, len(cond.Table.Info.Columns)) + for _, col := range cond.Table.ColumnMap { + columnNames = append(columnNames, dbutil.ColumnName(col.Name.O)) + } + columns := strings.Join(columnNames, ", ") + rowsQuery := fmt.Sprintf("SELECT /*!40001 SQL_NO_CACHE */ %s FROM %s WHERE %s ORDER BY %s", + columns, fullTableName, cond.GetWhere(), strings.Join(orderKeys, ",")) + rows, err := conn.QuerySQL(tctx, rowsQuery, cond.GetArgs()...) + if err != nil { + return nil, err + } + newRowDataIter := &RowDataIteratorImpl{ + Rows: rows, + } + return newRowDataIter, nil +} + +func (v *DataValidator) getRowChangeIterator(table *TableDiff, rows []*rowChange) (RowDataIterator, error) { + sort.Slice(rows, func(i, j int) bool { + left, right := rows[i], rows[j] + for idx := range left.pk { + if left.pk[idx] != right.pk[idx] { + return left.pk[idx] < right.pk[idx] + } + } + return false + }) + it := &RowChangeIteratorImpl{} + for _, r := range rows { + colMap := make(map[string]*dbutil.ColumnData) + for _, c := range table.Info.Columns { + var colData []byte + if r.data[c.Offset] != nil { + colData = []byte(fmt.Sprintf("%v", r.data[c.Offset])) + } + colMap[c.Name.O] = &dbutil.ColumnData{ + Data: colData, + IsNull: r.data[c.Offset] == nil, + } + } + it.Rows = append(it.Rows, colMap) + } + return it, nil +} + +func getPkValues(data map[string]*dbutil.ColumnData, cond *Cond) []string { + var pkValues []string + for _, pkColumn := range cond.Table.PrimaryKey.Columns { + // TODO primary key cannot be null, if we uses unique key should make sure all columns are not null + pkValues = append(pkValues, string(data[pkColumn.Name.O].Data)) + } + return pkValues +} + +func (v *DataValidator) CompareData(map1, map2 map[string]*dbutil.ColumnData, orderKeyCols, columns []*model.ColumnInfo) (equal bool, cmp int32, err error) { + var ( + data1, data2 *dbutil.ColumnData + str1, str2 string + key string + ok bool + ) + + equal = true + + defer func() { + if equal || err != nil { + return + } + + if cmp == 0 { + v.L.Warn("find different row", zap.String("column", key), zap.String("row1", rowToString(map1)), zap.String("row2", rowToString(map2))) + } else if cmp > 0 { + v.L.Warn("target had superfluous data", zap.String("row", rowToString(map2))) + } else { + v.L.Warn("target lack data", zap.String("row", rowToString(map1))) + } + }() + + for _, column := range columns { + if data1, ok = map1[column.Name.O]; !ok { + return false, 0, errors.Errorf("upstream don't have key %s", key) + } + if data2, ok = map2[column.Name.O]; !ok { + return false, 0, errors.Errorf("downstream don't have key %s", key) + } + str1 = string(data1.Data) + str2 = string(data2.Data) + if column.FieldType.Tp == tidbmysql.TypeFloat || column.FieldType.Tp == tidbmysql.TypeDouble { + if data1.IsNull == data2.IsNull && data1.IsNull { + continue + } + + num1, err1 := strconv.ParseFloat(str1, 64) + num2, err2 := strconv.ParseFloat(str2, 64) + if err1 != nil || err2 != nil { + err = errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", str1, str2, err1, err2) + return + } + if math.Abs(num1-num2) <= 1e-6 { + continue + } + } else { + if (str1 == str2) && (data1.IsNull == data2.IsNull) { + continue + } + } + + equal = false + break + + } + if equal { + return + } + + // Not Equal. Compare orderkeycolumns. + for _, col := range orderKeyCols { + if data1, ok = map1[col.Name.O]; !ok { + err = errors.Errorf("don't have key %s", col.Name.O) + return + } + if data2, ok = map2[col.Name.O]; !ok { + err = errors.Errorf("don't have key %s", col.Name.O) + return + } + + if NeedQuotes(col.FieldType.Tp) { + strData1 := string(data1.Data) + strData2 := string(data2.Data) + + if len(strData1) == len(strData2) && strData1 == strData2 { + continue + } + + if strData1 < strData2 { + cmp = -1 + } else { + cmp = 1 + } + break + } else if data1.IsNull || data2.IsNull { + if data1.IsNull && data2.IsNull { + continue + } + + if data1.IsNull { + cmp = -1 + } else { + cmp = 1 + } + break + } else { + num1, err1 := strconv.ParseFloat(string(data1.Data), 64) + num2, err2 := strconv.ParseFloat(string(data2.Data), 64) + if err1 != nil || err2 != nil { + err = errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", string(data1.Data), string(data2.Data), err1, err2) + return + } + + if num1 == num2 { + continue + } + + if num1 < num2 { + cmp = -1 + } else { + cmp = 1 + } + break + } + } + + return +} + +func NeedQuotes(tp byte) bool { + return !(dbutil.IsNumberType(tp) || dbutil.IsFloatType(tp)) +} + +func rowToString(row map[string]*dbutil.ColumnData) string { + var s strings.Builder + s.WriteString("{ ") + for key, val := range row { + if val.IsNull { + s.WriteString(fmt.Sprintf("%s: IsNull, ", key)) + } else { + s.WriteString(fmt.Sprintf("%s: %s, ", key, val.Data)) + } + } + s.WriteString(" }") + + return s.String() +} diff --git a/dm/syncer/validator_cond.go b/dm/syncer/validator_cond.go new file mode 100644 index 00000000000..a18d0be13dc --- /dev/null +++ b/dm/syncer/validator_cond.go @@ -0,0 +1,59 @@ +package syncer + +import ( + "strings" + + "github.com/pingcap/tidb-tools/pkg/dbutil" +) + +type Cond struct { + Table *TableDiff + PkValues [][]string +} + +func (c *Cond) GetArgs() []interface{} { + var res []interface{} + for _, v := range c.PkValues { + for _, val := range v { + res = append(res, val) + } + } + return res +} + +func (c *Cond) GetWhere() string { + var b strings.Builder + pk := c.Table.PrimaryKey + if len(pk.Columns) > 1 { + // TODO + panic("should be one") + } + b.WriteString(pk.Columns[0].Name.O) + b.WriteString(" in (") + for i := range c.PkValues { + if i != 0 { + b.WriteString(", ") + } + b.WriteString("?") + } + b.WriteString(")") + return b.String() +} + +type SimpleRowsIterator struct { + Rows []map[string]*dbutil.ColumnData + Idx int +} + +func (b *SimpleRowsIterator) Next() (map[string]*dbutil.ColumnData, error) { + if b.Idx >= len(b.Rows) { + return nil, nil + } + row := b.Rows[b.Idx] + b.Idx++ + return row, nil +} + +func (b *SimpleRowsIterator) Close() { + // skip +} From a67c580bb78ddaf9238dcdb66cf697dfa33a8c1a Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Wed, 9 Feb 2022 20:57:34 +0800 Subject: [PATCH 02/36] feat: cond ut --- dm/syncer/validator_cond.go | 34 ++++++++-- dm/syncer/validator_cond_test.go | 104 +++++++++++++++++++++++++++++++ 2 files changed, 132 insertions(+), 6 deletions(-) create mode 100644 dm/syncer/validator_cond_test.go diff --git a/dm/syncer/validator_cond.go b/dm/syncer/validator_cond.go index a18d0be13dc..24b02be10e1 100644 --- a/dm/syncer/validator_cond.go +++ b/dm/syncer/validator_cond.go @@ -1,3 +1,16 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package syncer import ( @@ -24,17 +37,26 @@ func (c *Cond) GetArgs() []interface{} { func (c *Cond) GetWhere() string { var b strings.Builder pk := c.Table.PrimaryKey - if len(pk.Columns) > 1 { - // TODO - panic("should be one") + b.WriteString("(") + for i := 0; i < len(pk.Columns); i++ { + if i != 0 { + b.WriteString(",") + } + b.WriteString(pk.Columns[i].Name.O) } - b.WriteString(pk.Columns[0].Name.O) - b.WriteString(" in (") + b.WriteString(") in (") for i := range c.PkValues { if i != 0 { b.WriteString(", ") } - b.WriteString("?") + b.WriteString("(") + for j := 0; j < len(pk.Columns); j++ { + if j != 0 { + b.WriteString(",") + } + b.WriteString("?") + } + b.WriteString(")") } b.WriteString(")") return b.String() diff --git a/dm/syncer/validator_cond_test.go b/dm/syncer/validator_cond_test.go new file mode 100644 index 00000000000..a01f148e297 --- /dev/null +++ b/dm/syncer/validator_cond_test.go @@ -0,0 +1,104 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package syncer + +import ( + "database/sql" + "fmt" + "strconv" + + sqlmock "github.com/DATA-DOG/go-sqlmock" + . "github.com/pingcap/check" + "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/model" +) + +type testCondSuite struct{} + +var _ = Suite(&testCondSuite{}) + +func getTableDiff(c *C, db *sql.DB, schemaName, tableName, creatSQL string) *TableDiff { + var ( + err error + parser2 *parser.Parser + tableInfo *model.TableInfo + ) + // ctx := context.Background() + // parser2, err = utils.GetParser(ctx, db) + parser2 = parser.New() + c.Assert(err, IsNil) + tableInfo, err = dbutil.GetTableInfoBySQL(creatSQL, parser2) + c.Assert(err, IsNil) + columnMap := make(map[string]*model.ColumnInfo) + for _, col := range tableInfo.Columns { + columnMap[col.Name.O] = col + } + var primaryIdx *model.IndexInfo + for _, idx := range tableInfo.Indices { + if idx.Primary { + primaryIdx = idx + } + } + tableDiff := &TableDiff{ + Schema: "test_cond", + Table: "test1", + Info: tableInfo, + PrimaryKey: primaryIdx, + ColumnMap: columnMap, + } + return tableDiff +} + +func (s *testCondSuite) TestCondSelectMultiKey(c *C) { + var ( + res *sql.Rows + ) + db, mock, err := sqlmock.New() + c.Assert(err, IsNil) + defer db.Close() + creatTbl := "create table if not exists `test_cond`.`test1`(" + + "a int," + + "b int," + + "c int," + + "primary key(a, b)" + + ");" + // get table diff + tableDiff := getTableDiff(c, db, "test_cond", "test1", creatTbl) + pkValues := make([][]string, 0) + for i := 0; i < 3; i++ { + // 3 primary key + key1, key2 := strconv.Itoa(i+1), strconv.Itoa(i+2) + pkValues = append(pkValues, []string{key1, key2}) + } + cond := &Cond{ + Table: tableDiff, + PkValues: pkValues, + } + // format query string + rowsQuery := fmt.Sprintf("SELECT COUNT(*) FROM %s WHERE %s;", "`test_cond`.`test1`", cond.GetWhere()) + mock.ExpectPrepare("SELECT COUNT\\(\\*\\) FROM `test_cond`.`test1` WHERE \\(a,b\\) in \\(\\(\\?,\\?\\), \\(\\?,\\?\\), \\(\\?,\\?\\)\\);").ExpectQuery().WithArgs( + "1", "2", "2", "3", "3", "4", + ).WillReturnRows(mock.NewRows([]string{"COUNT(*)"}).AddRow("3")) + prepare, err := db.Prepare(rowsQuery) + c.Assert(err, IsNil) + res, err = prepare.Query(cond.GetArgs()...) + c.Assert(err, IsNil) + var cnt int + if res.Next() { + err = res.Scan(&cnt) + } + c.Assert(err, IsNil) + c.Assert(cnt, Equals, 3) +} From cf136e0bd309f9c5788db2037dc55969e7a3b8fe Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Thu, 10 Feb 2022 19:31:12 +0800 Subject: [PATCH 03/36] feat: validator unit test --- dm/syncer/data_validator.go | 60 ++-- dm/syncer/data_validator_test.go | 462 +++++++++++++++++++++++++++++++ dm/syncer/validator_cond.go | 47 ++-- dm/syncer/validator_cond_test.go | 87 +++++- 4 files changed, 576 insertions(+), 80 deletions(-) create mode 100644 dm/syncer/data_validator_test.go diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index e394478996a..60d54c20ec1 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -177,7 +177,6 @@ type DataValidator struct { workerCnt int pendingChangeChs map[int]chan map[string]*tableChange // replace pendingChangeCh failedChangesMap map[int]map[string]*tableChange // replace failedChanges - workerLocks map[int]*sync.RWMutex // lock for each worker } func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *DataValidator { @@ -232,7 +231,6 @@ func (v *DataValidator) initialize() error { for i := 0; i < v.workerCnt; i++ { v.pendingChangeChs[i] = make(chan map[string]*tableChange) v.failedChangesMap[i] = make(map[string]*tableChange) - v.workerLocks[i] = &sync.RWMutex{} } return nil } @@ -304,10 +302,6 @@ func (v *DataValidator) doValidate() { v.wg.Add(2) go v.rowsEventProcessRoutine() go v.validateTaskDispatchRoutine() - v.wg.Add(v.workerCnt) - for i := 0; i < v.workerCnt; i++ { - go v.retryFailedRowsByWorker(i) - } var latestPos mysql.Position for { e, err := v.streamerController.GetEvent(tctx) @@ -369,24 +363,6 @@ func (v *DataValidator) Stage() pb.Stage { return v.stage } -func (v *DataValidator) retryFailedRowsByWorker(workerID int) { - v.wg.Done() - for { - v.workerLocks[workerID].Lock() - retryChange := v.failedChangesMap[workerID] - failed := v.validateTableChange(retryChange) - deltaFailed := v.updateFailedChangesByWorker(retryChange, failed, workerID) - v.failedRowCnt.Add(int64(deltaFailed)) - v.workerLocks[workerID].Unlock() - // todo: print validation summary - select { - case <-v.ctx.Done(): - return - case <-time.After(retryInterval): - } - } -} - func (v *DataValidator) rowsEventProcessRoutine() { v.wg.Done() for { @@ -413,7 +389,6 @@ func (v *DataValidator) workerValidateTableChange(workerID int) { for { select { case change := <-v.pendingChangeChs[workerID]: - v.workerLocks[workerID].Lock() // 1. validate table change // 2. update failed rows // 3. update pending row count @@ -422,11 +397,14 @@ func (v *DataValidator) workerValidateTableChange(workerID int) { deltaFailed := v.updateFailedChangesByWorker(change, failed, workerID) v.pendingRowCnt.Sub(int64(v.getRowCount(change))) // atomic value v.failedRowCnt.Add(int64(deltaFailed)) // atomic value - v.workerLocks[workerID].Unlock() case <-v.ctx.Done(): return + case <-time.After(retryInterval): + retryChange := v.failedChangesMap[workerID] + failed := v.validateTableChange(retryChange) + deltaFailed := v.updateFailedChangesByWorker(retryChange, failed, workerID) + v.failedRowCnt.Add(int64(deltaFailed)) } - } } @@ -715,8 +693,9 @@ func (v *DataValidator) updateFailedChangesByWorker(all, failed map[string]*tabl } func (v *DataValidator) validateDeletedRows(cond *Cond) ([][]string, error) { - downstreamRowsIterator, err := v.getRowsFrom(cond, v.toDBConn) + downstreamRowsIterator, err := getRowsFrom(cond, v.toDBConn) if err != nil { + // todo: cancel all routine return nil, errors.Trace(err) } defer downstreamRowsIterator.Close() @@ -730,7 +709,7 @@ func (v *DataValidator) validateDeletedRows(cond *Cond) ([][]string, error) { if data == nil { break } - failedRows = append(failedRows, getPkValues(data, cond)) + failedRows = append(failedRows, getPKValues(data, cond)) } return failedRows, nil } @@ -739,13 +718,14 @@ func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Con var failedRows [][]string var upstreamRowsIterator, downstreamRowsIterator RowDataIterator var err error - upstreamRowsIterator, err = v.getRowChangeIterator(cond.Table, rows) + upstreamRowsIterator, err = getRowChangeIterator(cond.Table, rows) if err != nil { return nil, errors.New("") } defer upstreamRowsIterator.Close() - downstreamRowsIterator, err = v.getRowsFrom(cond, v.toDBConn) + downstreamRowsIterator, err = getRowsFrom(cond, v.toDBConn) if err != nil { + // todo: cancel all routine return nil, errors.New("") } defer downstreamRowsIterator.Close() @@ -780,7 +760,7 @@ func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Con if lastDownstreamData == nil { // target lack some data, should insert the last source datas for lastUpstreamData != nil { - failedRows = append(failedRows, getPkValues(lastUpstreamData, cond)) + failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) lastUpstreamData, err = upstreamRowsIterator.Next() if err != nil { @@ -790,7 +770,7 @@ func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Con break } - eq, cmp, err := v.CompareData(lastUpstreamData, lastDownstreamData, orderKeyCols, tableInfo.Columns) + eq, cmp, err := v.compareData(lastUpstreamData, lastDownstreamData, orderKeyCols, tableInfo.Columns) if err != nil { return nil, errors.New("") } @@ -808,10 +788,10 @@ func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Con v.L.Debug("more data on downstream, may come from other client, skip it", zap.Reflect("data", lastDownstreamData)) lastDownstreamData = nil case -1: - failedRows = append(failedRows, getPkValues(lastUpstreamData, cond)) + failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) lastUpstreamData = nil case 0: - failedRows = append(failedRows, getPkValues(lastUpstreamData, cond)) + failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) lastUpstreamData = nil lastDownstreamData = nil } @@ -819,8 +799,8 @@ func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Con return failedRows, nil } -func (v *DataValidator) getRowsFrom(cond *Cond, conn *dbconn.DBConn) (RowDataIterator, error) { - tctx := tcontext.NewContext(v.ctx, v.L) +func getRowsFrom(cond *Cond, conn *dbconn.DBConn) (RowDataIterator, error) { + tctx := tcontext.NewContext(context.Background(), log.L()) fullTableName := dbutil.TableName(cond.Table.Schema, cond.Table.Table) orderKeys, _ := dbutil.SelectUniqueOrderKey(cond.Table.Info) columnNames := make([]string, 0, len(cond.Table.Info.Columns)) @@ -840,7 +820,7 @@ func (v *DataValidator) getRowsFrom(cond *Cond, conn *dbconn.DBConn) (RowDataIte return newRowDataIter, nil } -func (v *DataValidator) getRowChangeIterator(table *TableDiff, rows []*rowChange) (RowDataIterator, error) { +func getRowChangeIterator(table *TableDiff, rows []*rowChange) (RowDataIterator, error) { sort.Slice(rows, func(i, j int) bool { left, right := rows[i], rows[j] for idx := range left.pk { @@ -868,7 +848,7 @@ func (v *DataValidator) getRowChangeIterator(table *TableDiff, rows []*rowChange return it, nil } -func getPkValues(data map[string]*dbutil.ColumnData, cond *Cond) []string { +func getPKValues(data map[string]*dbutil.ColumnData, cond *Cond) []string { var pkValues []string for _, pkColumn := range cond.Table.PrimaryKey.Columns { // TODO primary key cannot be null, if we uses unique key should make sure all columns are not null @@ -877,7 +857,7 @@ func getPkValues(data map[string]*dbutil.ColumnData, cond *Cond) []string { return pkValues } -func (v *DataValidator) CompareData(map1, map2 map[string]*dbutil.ColumnData, orderKeyCols, columns []*model.ColumnInfo) (equal bool, cmp int32, err error) { +func (v *DataValidator) compareData(map1, map2 map[string]*dbutil.ColumnData, orderKeyCols, columns []*model.ColumnInfo) (equal bool, cmp int32, err error) { var ( data1, data2 *dbutil.ColumnData str1, str2 string diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go new file mode 100644 index 00000000000..9d410f8c9f4 --- /dev/null +++ b/dm/syncer/data_validator_test.go @@ -0,0 +1,462 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package syncer + +import ( + "context" + "database/sql" + "database/sql/driver" + "os" + "sort" + + "github.com/DATA-DOG/go-sqlmock" + "github.com/go-mysql-org/go-mysql/mysql" + . "github.com/pingcap/check" + "github.com/pingcap/tidb-tools/pkg/dbutil" + parsermysql "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/syncer/dbconn" +) + +type testDataValidatorSuite struct { + eventsGenerator *event.Generator + cfg *config.SubTaskConfig +} + +var _ = Suite(&testDataValidatorSuite{}) + +func (d *testDataValidatorSuite) SetUpSuite(c *C) { + previousGTIDSetStr := "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-14,406a3f61-690d-11e7-87c5-6c92bf46f384:1-94321383" + previousGTIDSet, err := gtid.ParserGTID(mysql.MySQLFlavor, previousGTIDSetStr) + if err != nil { + c.Fatal(err) + } + latestGTIDStr := "3ccc475b-2343-11e7-be21-6c0b84d59f30:14" + latestGTID, err := gtid.ParserGTID(mysql.MySQLFlavor, latestGTIDStr) + c.Assert(err, IsNil) + d.eventsGenerator, err = event.NewGenerator(mysql.MySQLFlavor, 1, 0, latestGTID, previousGTIDSet, 0) + if err != nil { + c.Fatal(err) + } + loaderDir, err := os.MkdirTemp("", "loader") + c.Assert(err, IsNil) + loaderCfg := config.LoaderConfig{ + Dir: loaderDir, + } + d.cfg = &config.SubTaskConfig{ + From: config.GetDBConfigForTest(), + To: config.GetDBConfigForTest(), + ServerID: 101, + MetaSchema: "test", + Name: "syncer_ut", + ShadowTableRules: []string{config.DefaultShadowTableRules}, + TrashTableRules: []string{config.DefaultTrashTableRules}, + Mode: config.ModeIncrement, + Flavor: "mysql", + LoaderConfig: loaderCfg, + } + d.cfg.Experimental.AsyncCheckpointFlush = true + d.cfg.From.Adjust() + d.cfg.To.Adjust() + + d.cfg.UseRelay = false +} + +func (d *testDataValidatorSuite) genDBConn(c *C, db *sql.DB) *dbconn.DBConn { + baseDB := conn.NewBaseDB(db, func() {}) + baseConn, err := baseDB.GetBaseConn(context.Background()) + c.Assert(err, IsNil) + cfg, err := d.cfg.Clone() + c.Assert(err, IsNil) + return &dbconn.DBConn{ + BaseConn: baseConn, + Cfg: cfg, + } +} + +func (d *testDataValidatorSuite) TestRowDataIteratorImpl(c *C) { + var ( + iter RowDataIterator + dbConn *dbconn.DBConn + ) + db, mock, err := sqlmock.New() + c.Assert(err, IsNil) + type testTableSchm struct { + a string + b string + } + rowVals := []testTableSchm{ + { + a: "a", + b: "100", + }, + { + a: "c", + b: "200", + }, + { + a: "d", + b: "300", + }, + { + a: "e", + b: "400", + }, + } + expectRows := sqlmock.NewRows([]string{"a", "b"}) + for i := 0; i < len(rowVals); i++ { + expectRows = expectRows.AddRow(rowVals[i].a, rowVals[i].b) + } + mock.ExpectQuery( + "SELECT .* FROM .* WHERE a in .*", + ).WithArgs( + "a", "c", "d", "e", + ).WillReturnRows(expectRows) + c.Assert(err, IsNil) + cond := formatCond(c, db, "test_row", "test", ` + create table if not exists test_row.test( + a char(1), + b int, + primary key(a) + ); + `, [][]string{{"a"}, {"c"}, {"d"}, {"e"}}) + c.Assert(err, IsNil) + dbConn = d.genDBConn(c, db) + iter, err = getRowsFrom(cond, dbConn) + c.Assert(err, IsNil) + for i := range rowVals { + curVal, err := iter.Next() + c.Assert(err, IsNil) + keys := make([]string, 0) + for key := range curVal { + keys = append(keys, key) + } + sort.Slice(keys, func(i, j int) bool { + return keys[i] < keys[j] + }) + c.Assert(keys[0], Equals, "a") + c.Assert(keys[1], Equals, "b") + c.Assert(string(curVal["a"].Data), Equals, rowVals[i].a) + c.Assert(string(curVal["b"].Data), Equals, rowVals[i].b) + } + nxtVal, err := iter.Next() + c.Assert(nxtVal, IsNil) + c.Assert(err, IsNil) +} + +func (r *testDataValidatorSuite) TestRowChangeIteratorImpl(c *C) { + db, _, err := sqlmock.New() + c.Assert(err, IsNil) + rows := []*rowChange{ + { + pk: []string{"pk", "1"}, + data: []interface{}{"pk", "1", "some data"}, + theType: rowInsert, + }, + { + pk: []string{"pk", "2"}, + data: []interface{}{"pk", "2", "some data"}, + theType: rowDeleted, + }, + { + pk: []string{"pkg", "2"}, + data: []interface{}{"pkg", "2", "some data"}, + theType: rowDeleted, + }, + { + pk: []string{"pke", "2"}, + data: []interface{}{"pke", "2", "some data"}, + theType: rowUpdated, + }, + } + testCases := []map[string]string{ + { + "pk1": "pk", + "pk2": "1", + "other": "some data", + }, + { + "pk1": "pk", + "pk2": "2", + "other": "some data", + }, + { + "pk1": "pke", + "pk2": "2", + "other": "some data", + }, + { + "pk1": "pkg", + "pk2": "2", + "other": "some data", + }, + } + creatTbl := `create table if not exists test_it.test1 ( + pk1 varchar(4), + pk2 int, + other text, + primary key(pk1, pk2) + );` + tableDiff := getTableDiff(c, db, "test_it", "test1", creatTbl) + var iter RowDataIterator + iter, err = getRowChangeIterator(tableDiff, rows) + c.Assert(err, IsNil) + for i := 0; i < len(rows); i++ { + res, err := iter.Next() + c.Assert(err, IsNil) + c.Assert(res, NotNil) + colNames := []string{} + for key := range res { + colNames = append(colNames, key) + c.Assert(string(res[key].Data), Equals, testCases[i][key]) + } + sort.Slice(colNames, func(left, right int) bool { + return colNames[left] < colNames[right] + }) + c.Assert(colNames[0], Equals, "other") + c.Assert(colNames[1], Equals, "pk1") + c.Assert(colNames[2], Equals, "pk2") + } +} + +func (d *testDataValidatorSuite) TestGetRowsFrom(c *C) { + type testCase struct { + schemaName string + tblName string + creatSQL string + pkValues [][]string + allCols []string + rowData [][]string + querySQL string + } + testCases := []testCase{ + { + schemaName: "test1", + tblName: "tbl1", + creatSQL: `create table if not exists test1.tbl1( + a int, + b int, + c int, + primary key(a, b) + );`, + pkValues: [][]string{ + {"1", "2"}, {"3", "4"}, {"5", "6"}, + }, + allCols: []string{"a", "b", "c"}, + rowData: [][]string{ + {"1", "2", "3"}, {"3", "4", "5"}, {"5", "6", "7"}, + }, + querySQL: "SELECT .* FROM .*test1.*", + }, + { + schemaName: "test2", + tblName: "tbl2", + creatSQL: `create table if not exists test2.tbl2( + a char(10), + other text, + primary key(a) + );`, + pkValues: [][]string{ + {"a"}, {"b"}, {"c"}, + }, + allCols: []string{"a", "other"}, + rowData: [][]string{ + {"a", "some data"}, {"b", "some data"}, {"c", "some data"}, + }, + querySQL: "SELECT .* FROM .*test2.*", + }, + } + db, mock, err := sqlmock.New() + c.Assert(err, IsNil) + for i, testCase := range testCases { + args := []driver.Value{} + for _, arr := range testCases[i].pkValues { + for _, val := range arr { + args = append(args, val) + } + } + dataRows := mock.NewRows(testCase.allCols) + for j := range testCases[i].rowData { + args := []driver.Value{} + for _, val := range testCase.rowData[j] { + args = append(args, val) + } + dataRows = dataRows.AddRow(args...) + } + mock.ExpectQuery(testCase.querySQL).WithArgs(args...).WillReturnRows(dataRows) + cond := formatCond( + c, db, + testCase.schemaName, + testCase.tblName, + testCase.creatSQL, + testCase.pkValues, + ) + dbConn := d.genDBConn(c, db) + var iter RowDataIterator + iter, err = getRowsFrom(cond, dbConn) + c.Assert(err, IsNil) + var res map[string]*dbutil.ColumnData + for j := 0; j < 3; j++ { + res, err = iter.Next() + c.Assert(err, IsNil) + pkRes := getPKValues(res, cond) + // for each primary key + for k, val := range testCase.pkValues[j] { + c.Assert(pkRes[k], Equals, val) + } + // for each col + for k, val := range testCase.rowData[j] { + colName := testCase.allCols[k] + c.Assert(string(res[colName].Data), Equals, val) + } + } + res, err = iter.Next() + c.Assert(res, IsNil) + c.Assert(err, IsNil) + } +} + +func (d *testDataValidatorSuite) TestRowToString(c *C) { + testCases := []map[string]*dbutil.ColumnData{ + { + "pk-1": &dbutil.ColumnData{ + Data: []byte("some data"), + IsNull: false, + }, + "pk-2": &dbutil.ColumnData{ + Data: []byte("simple data"), + IsNull: false, + }, + }, + { + "pk-3": &dbutil.ColumnData{ + Data: nil, + IsNull: true, + }, + "pk-4": &dbutil.ColumnData{ + Data: []byte("data"), + IsNull: false, + }, + }, + } + expectedStr := []string{ + "{ pk-1: some data, pk-2: simple data, }", + "{ pk-3: IsNull, pk-4: data, }", + } + for i, testCase := range testCases { + ret := rowToString(testCase) + c.Assert(ret, Equals, expectedStr[i]) + } +} + +func (d *testDataValidatorSuite) TestNeedQuotes(c *C) { + testCases := map[byte]bool{ + parsermysql.TypeTiny: false, + parsermysql.TypeShort: false, + parsermysql.TypeLong: false, + parsermysql.TypeLonglong: false, + parsermysql.TypeInt24: false, + parsermysql.TypeYear: false, + parsermysql.TypeFloat: false, + parsermysql.TypeDouble: false, + parsermysql.TypeNewDecimal: false, + parsermysql.TypeDuration: true, + parsermysql.TypeDatetime: true, + parsermysql.TypeNewDate: true, + parsermysql.TypeVarchar: true, + parsermysql.TypeBlob: true, + parsermysql.TypeVarString: true, + parsermysql.TypeString: true, + } + for typ, val := range testCases { + c.Assert(NeedQuotes(typ), Equals, val) + } +} + +func (d *testDataValidatorSuite) TestGetPKValues(c *C) { + testCases := []map[string]*dbutil.ColumnData{ + { + "col1": &dbutil.ColumnData{ + Data: []byte("some data"), + }, + "col2": &dbutil.ColumnData{ + Data: []byte("data"), + }, + "col3": &dbutil.ColumnData{ + IsNull: true, + }, + }, + { + "c1": &dbutil.ColumnData{ + Data: []byte("lk"), + }, + "c2": &dbutil.ColumnData{ + Data: []byte("1001"), + }, + }, + } + expectedOut := [][]string{ + {"some data"}, + {"lk", "1001"}, + } + type testTable struct { + schemaName string + tableName string + creatSQL string + } + testTables := []testTable{ + { + schemaName: "test1", + tableName: "tbl1", + creatSQL: `create table if not exists test1.tbl1( + col1 varchar(10), + col2 text, + col3 varchar(20), + primary key(col1) + );`, + }, + { + schemaName: "test2", + tableName: "tbl2", + creatSQL: `create table if not exists test2.tbl2( + c1 char(2), + c2 int, + primary key(c1, c2) + );`, + }, + } + db, _, err := sqlmock.New() + c.Assert(err, IsNil) + for i := range testCases { + cond := formatCond( + c, + db, + testTables[i].schemaName, + testTables[i].tableName, + testTables[i].creatSQL, + [][]string{}, + ) + ret := getPKValues(testCases[i], cond) + for j, val := range expectedOut[i] { + c.Assert(ret[j], Equals, val) + } + } +} + +func (d *testDataValidatorSuite) TestDoValidate(c *C) { + +} diff --git a/dm/syncer/validator_cond.go b/dm/syncer/validator_cond.go index 24b02be10e1..3c6b5865162 100644 --- a/dm/syncer/validator_cond.go +++ b/dm/syncer/validator_cond.go @@ -15,8 +15,6 @@ package syncer import ( "strings" - - "github.com/pingcap/tidb-tools/pkg/dbutil" ) type Cond struct { @@ -37,45 +35,38 @@ func (c *Cond) GetArgs() []interface{} { func (c *Cond) GetWhere() string { var b strings.Builder pk := c.Table.PrimaryKey - b.WriteString("(") + isOneKey := len(pk.Columns) == 1 + if !isOneKey { + b.WriteString("(") + } for i := 0; i < len(pk.Columns); i++ { if i != 0 { b.WriteString(",") } b.WriteString(pk.Columns[i].Name.O) } - b.WriteString(") in (") + if !isOneKey { + b.WriteString(") in (") + } else { + b.WriteString(" in (") + } for i := range c.PkValues { if i != 0 { - b.WriteString(", ") + b.WriteString(",") } - b.WriteString("(") - for j := 0; j < len(pk.Columns); j++ { - if j != 0 { - b.WriteString(",") + if !isOneKey { + b.WriteString("(") + for j := 0; j < len(pk.Columns); j++ { + if j != 0 { + b.WriteString(",") + } + b.WriteString("?") } + b.WriteString(")") + } else { b.WriteString("?") } - b.WriteString(")") } b.WriteString(")") return b.String() } - -type SimpleRowsIterator struct { - Rows []map[string]*dbutil.ColumnData - Idx int -} - -func (b *SimpleRowsIterator) Next() (map[string]*dbutil.ColumnData, error) { - if b.Idx >= len(b.Rows) { - return nil, nil - } - row := b.Rows[b.Idx] - b.Idx++ - return row, nil -} - -func (b *SimpleRowsIterator) Close() { - // skip -} diff --git a/dm/syncer/validator_cond_test.go b/dm/syncer/validator_cond_test.go index a01f148e297..88a8e8bf0f2 100644 --- a/dm/syncer/validator_cond_test.go +++ b/dm/syncer/validator_cond_test.go @@ -35,8 +35,6 @@ func getTableDiff(c *C, db *sql.DB, schemaName, tableName, creatSQL string) *Tab parser2 *parser.Parser tableInfo *model.TableInfo ) - // ctx := context.Background() - // parser2, err = utils.GetParser(ctx, db) parser2 = parser.New() c.Assert(err, IsNil) tableInfo, err = dbutil.GetTableInfoBySQL(creatSQL, parser2) @@ -52,8 +50,8 @@ func getTableDiff(c *C, db *sql.DB, schemaName, tableName, creatSQL string) *Tab } } tableDiff := &TableDiff{ - Schema: "test_cond", - Table: "test1", + Schema: schemaName, + Table: tableName, Info: tableInfo, PrimaryKey: primaryIdx, ColumnMap: columnMap, @@ -61,6 +59,14 @@ func getTableDiff(c *C, db *sql.DB, schemaName, tableName, creatSQL string) *Tab return tableDiff } +func formatCond(c *C, db *sql.DB, schemaName, tblName, creatSQL string, pkvs [][]string) *Cond { + tblDiff := getTableDiff(c, db, schemaName, tblName, creatSQL) + return &Cond{ + Table: tblDiff, + PkValues: pkvs, + } +} + func (s *testCondSuite) TestCondSelectMultiKey(c *C) { var ( res *sql.Rows @@ -75,25 +81,22 @@ func (s *testCondSuite) TestCondSelectMultiKey(c *C) { "primary key(a, b)" + ");" // get table diff - tableDiff := getTableDiff(c, db, "test_cond", "test1", creatTbl) pkValues := make([][]string, 0) for i := 0; i < 3; i++ { // 3 primary key key1, key2 := strconv.Itoa(i+1), strconv.Itoa(i+2) pkValues = append(pkValues, []string{key1, key2}) } - cond := &Cond{ - Table: tableDiff, - PkValues: pkValues, - } + cond := formatCond(c, db, "test_cond", "test1", creatTbl, pkValues) // format query string rowsQuery := fmt.Sprintf("SELECT COUNT(*) FROM %s WHERE %s;", "`test_cond`.`test1`", cond.GetWhere()) - mock.ExpectPrepare("SELECT COUNT\\(\\*\\) FROM `test_cond`.`test1` WHERE \\(a,b\\) in \\(\\(\\?,\\?\\), \\(\\?,\\?\\), \\(\\?,\\?\\)\\);").ExpectQuery().WithArgs( + mock.ExpectQuery( + "SELECT COUNT\\(\\*\\) FROM `test_cond`.`test1` WHERE \\(a,b\\) in \\(\\(\\?,\\?\\),\\(\\?,\\?\\),\\(\\?,\\?\\)\\);", + ).WithArgs( "1", "2", "2", "3", "3", "4", ).WillReturnRows(mock.NewRows([]string{"COUNT(*)"}).AddRow("3")) - prepare, err := db.Prepare(rowsQuery) c.Assert(err, IsNil) - res, err = prepare.Query(cond.GetArgs()...) + res, err = db.Query(rowsQuery, cond.GetArgs()...) c.Assert(err, IsNil) var cnt int if res.Next() { @@ -102,3 +105,63 @@ func (s *testCondSuite) TestCondSelectMultiKey(c *C) { c.Assert(err, IsNil) c.Assert(cnt, Equals, 3) } + +func (s *testCondSuite) TestCondGetWhereArgs(c *C) { + db, _, err := sqlmock.New() + c.Assert(err, IsNil) + defer db.Close() + type testCase struct { + creatTbl string + pks [][]string + tblName string + schemaName string + args []string + where string + } + cases := []testCase{ + { + creatTbl: `create table if not exists test_cond.test2( + a char(10), + b int, + c int, + primary key(a) + );`, // single primary key, + pks: [][]string{ + {"10a0"}, {"200"}, {"abc"}, + }, + tblName: "test2", + schemaName: "test_cond", + where: "a in (?,?,?)", + args: []string{ + "10a0", "200", "abc", + }, + }, + { + creatTbl: `create table if not exists test_cond.test3( + a int, + b char(10), + c varchar(10), + primary key(a, b, c) + );`, // multi primary key + pks: [][]string{ + {"10", "abc", "ef"}, + {"9897", "afdkiefkjg", "acdee"}, + }, + tblName: "test3", + schemaName: "test_cond", + where: "(a,b,c) in ((?,?,?),(?,?,?))", + args: []string{ + "10", "abc", "ef", "9897", "afdkiefkjg", "acdee", + }, + }, + } + for i := 0; i < len(cases); i++ { + cond := formatCond(c, db, cases[i].schemaName, cases[i].tblName, cases[i].creatTbl, cases[i].pks) + c.Assert(cond.GetWhere(), Equals, cases[i].where) + rawArgs := cond.GetArgs() + for j := 0; j < 3; j++ { + curData := fmt.Sprintf("%v", rawArgs[j]) + c.Assert(curData, Equals, cases[i].args[j]) + } + } +} From 3235bb3bfae53fbbd0031fd746ea06160a2325b5 Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Sat, 12 Feb 2022 01:33:48 +0800 Subject: [PATCH 04/36] feat: validator dovalidate ut --- dm/syncer/data_validator.go | 73 ++++++---- dm/syncer/data_validator_test.go | 233 ++++++++++++++++++++++++++++++- 2 files changed, 277 insertions(+), 29 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 60d54c20ec1..bbdcf1817a8 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -164,7 +164,9 @@ type DataValidator struct { syncCfg replication.BinlogSyncerConfig streamerController *StreamerController - result pb.ProcessResult + result pb.ProcessResult + batchRowCnt int + retryInterval time.Duration failedRowCnt atomic.Int64 accumulatedChanges map[string]*tableChange @@ -201,9 +203,24 @@ func (v *DataValidator) initialize() error { } }() + v.workerCnt = defaultWorkerCnt + v.pendingChangeChs = make(map[int]chan map[string]*tableChange) + v.failedChangesMap = make(map[int]map[string]*tableChange) + for i := 0; i < v.workerCnt; i++ { + v.pendingChangeChs[i] = make(chan map[string]*tableChange) + v.failedChangesMap[i] = make(map[string]*tableChange) + } + v.batchRowCnt = batchRowCount + v.validationTimer = time.NewTimer(validationInterval) + v.rowsEventChan = make(chan *replication.BinlogEvent) + v.pendingChangeCh = make(chan map[string]*tableChange) + v.diffTables = make(map[string]*TableDiff) + v.changeEventCount = make([]int, 4) + v.accumulatedChanges = make(map[string]*tableChange) + v.retryInterval = retryInterval + dbCfg := v.cfg.From dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxDMLConnectionTimeout) - // v.fromDB, err = dbconn.CreateBaseDB(&dbCfg) var fromDBConns, toDBConns []*dbconn.DBConn v.fromDB, fromDBConns, err = dbconn.CreateConns(tctx, v.cfg, &dbCfg, 1) if err != nil { @@ -226,12 +243,6 @@ func (v *DataValidator) initialize() error { } v.streamerController = NewStreamerController(v.syncCfg, v.cfg.EnableGTID, &dbconn.UpStreamConn{BaseDB: v.fromDB}, v.cfg.RelayDir, v.timezone, nil) - - v.workerCnt = defaultWorkerCnt - for i := 0; i < v.workerCnt; i++ { - v.pendingChangeChs[i] = make(chan map[string]*tableChange) - v.failedChangesMap[i] = make(map[string]*tableChange) - } return nil } @@ -292,10 +303,12 @@ func (v *DataValidator) fillResult(err error, needLock bool) { func (v *DataValidator) doValidate() { tctx := tcontext.NewContext(v.ctx, v.L) - err := v.streamerController.Start(tctx, lastLocation) - if err != nil { - v.fillResult(terror.Annotate(err, "fail to restart streamer controller"), true) - return + if v.streamerController.IsClosed() { + err := v.streamerController.Start(tctx, lastLocation) + if err != nil { + v.fillResult(terror.Annotate(err, "fail to restart streamer controller"), true) + return + } } v.L.Info("start continuous validation") @@ -306,7 +319,8 @@ func (v *DataValidator) doValidate() { for { e, err := v.streamerController.GetEvent(tctx) if err != nil { - v.L.Fatal("getting event from streamer controller failed") + // todo: validation paused? + v.fillResult(errors.New("getting binlog event failed"), true) return } // todo: configuring the time or use checkpoint @@ -364,10 +378,11 @@ func (v *DataValidator) Stage() pb.Stage { } func (v *DataValidator) rowsEventProcessRoutine() { - v.wg.Done() for { select { case <-v.ctx.Done(): + fmt.Printf("rowsEventProcessRoutine Done\n") + v.wg.Done() return case e := <-v.rowsEventChan: if err := v.processEventRows(e.Header, e.Event.(*replication.RowsEvent)); err != nil { @@ -385,7 +400,6 @@ func (v *DataValidator) rowsEventProcessRoutine() { } func (v *DataValidator) workerValidateTableChange(workerID int) { - v.wg.Done() for { select { case change := <-v.pendingChangeChs[workerID]: @@ -398,8 +412,10 @@ func (v *DataValidator) workerValidateTableChange(workerID int) { v.pendingRowCnt.Sub(int64(v.getRowCount(change))) // atomic value v.failedRowCnt.Add(int64(deltaFailed)) // atomic value case <-v.ctx.Done(): + fmt.Printf("workerValidateTableChange %d Done\n", workerID) + v.wg.Done() return - case <-time.After(retryInterval): + case <-time.After(v.retryInterval): retryChange := v.failedChangesMap[workerID] failed := v.validateTableChange(retryChange) deltaFailed := v.updateFailedChangesByWorker(retryChange, failed, workerID) @@ -409,7 +425,6 @@ func (v *DataValidator) workerValidateTableChange(workerID int) { } func (v *DataValidator) validateTaskDispatchRoutine() { - v.wg.Done() // start workers here v.wg.Add(v.workerCnt) for i := 0; i < v.workerCnt; i++ { @@ -420,6 +435,8 @@ func (v *DataValidator) validateTaskDispatchRoutine() { case change := <-v.pendingChangeCh: v.dispatchRowChange(change) // dispatch change to worker case <-v.ctx.Done(): + fmt.Printf("validateTaskDispatchRoutine Done\n") + v.wg.Done() return } } @@ -463,8 +480,11 @@ func (v *DataValidator) dispatchRowChange(change map[string]*tableChange) { func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *replication.RowsEvent) error { schemaName, tableName := string(ev.Table.Schema), string(ev.Table.Table) fullTableName := fmt.Sprintf("%s.%s", schemaName, tableName) - var table *TableDiff - if _, ok := v.diffTables[fullTableName]; !ok { + var ( + table *TableDiff + ok bool + ) + if table, ok = v.diffTables[fullTableName]; !ok { // table not found in cache // try getting table info from the upstream tctx := tcontext.NewContext(v.ctx, v.L) @@ -560,7 +580,7 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re val.theType = changeType val.lastMeetTs = int64(header.Timestamp) - if rowCount >= batchRowCount { + if rowCount >= v.batchRowCnt { v.pendingChangeCh <- v.accumulatedChanges v.accumulatedChanges = make(map[string]*tableChange) @@ -647,7 +667,8 @@ func (v *DataValidator) validateChanges(table *TableDiff, rows []*rowChange, del failedRows, err = v.validateInsertAndUpdateRows(rows, cond) } if err != nil { - panic(err) + v.fillResult(err, false) + return [][]string{} } return failedRows } @@ -695,8 +716,7 @@ func (v *DataValidator) updateFailedChangesByWorker(all, failed map[string]*tabl func (v *DataValidator) validateDeletedRows(cond *Cond) ([][]string, error) { downstreamRowsIterator, err := getRowsFrom(cond, v.toDBConn) if err != nil { - // todo: cancel all routine - return nil, errors.Trace(err) + return [][]string{}, err } defer downstreamRowsIterator.Close() @@ -720,13 +740,14 @@ func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Con var err error upstreamRowsIterator, err = getRowChangeIterator(cond.Table, rows) if err != nil { - return nil, errors.New("") + return nil, errors.New("get row change iter fails") } defer upstreamRowsIterator.Close() + fmt.Printf("cond: %v\n", cond) downstreamRowsIterator, err = getRowsFrom(cond, v.toDBConn) if err != nil { - // todo: cancel all routine - return nil, errors.New("") + fmt.Printf("get row fails: %s\n", err.Error()) + return nil, errors.New("get rows fails") } defer downstreamRowsIterator.Close() diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index 9d410f8c9f4..3fbbb0aba83 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -17,15 +17,20 @@ import ( "context" "database/sql" "database/sql/driver" + "fmt" "os" "sort" + "time" "github.com/DATA-DOG/go-sqlmock" "github.com/go-mysql-org/go-mysql/mysql" + "github.com/go-mysql-org/go-mysql/replication" . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/dbutil" parsermysql "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" "github.com/pingcap/tiflow/dm/pkg/binlog/event" "github.com/pingcap/tiflow/dm/pkg/conn" "github.com/pingcap/tiflow/dm/pkg/gtid" @@ -354,12 +359,12 @@ func (d *testDataValidatorSuite) TestRowToString(c *C) { }, } expectedStr := []string{ - "{ pk-1: some data, pk-2: simple data, }", - "{ pk-3: IsNull, pk-4: data, }", + "({ pk-1: some data, pk-2: simple data, }|{ pk-2: simple data, pk-1: some data, })", + "({ pk-3: IsNull, pk-4: data, }|{ pk-4: data, pk-3: IsNull, })", } for i, testCase := range testCases { ret := rowToString(testCase) - c.Assert(ret, Equals, expectedStr[i]) + c.Assert(ret, Matches, expectedStr[i]) } } @@ -457,6 +462,228 @@ func (d *testDataValidatorSuite) TestGetPKValues(c *C) { } } +func (d *testDataValidatorSuite) generateEvents(binlogEvents mockBinlogEvents, c *C) []*replication.BinlogEvent { + events := make([]*replication.BinlogEvent, 0, 1024) + for _, e := range binlogEvents { + switch e.typ { + case DBCreate: + evs, _, err := d.eventsGenerator.GenCreateDatabaseEvents(e.args[0].(string)) + c.Assert(err, IsNil) + events = append(events, evs...) + case DBDrop: + evs, _, err := d.eventsGenerator.GenDropDatabaseEvents(e.args[0].(string)) + c.Assert(err, IsNil) + events = append(events, evs...) + case TableCreate: + evs, _, err := d.eventsGenerator.GenCreateTableEvents(e.args[0].(string), e.args[1].(string)) + c.Assert(err, IsNil) + events = append(events, evs...) + case TableDrop: + evs, _, err := d.eventsGenerator.GenDropTableEvents(e.args[0].(string), e.args[1].(string)) + c.Assert(err, IsNil) + events = append(events, evs...) + + case DDL: + evs, _, err := d.eventsGenerator.GenDDLEvents(e.args[0].(string), e.args[1].(string), 0) + c.Assert(err, IsNil) + events = append(events, evs...) + + case Write, Update, Delete: + dmlData := []*event.DMLData{ + { + TableID: e.args[0].(uint64), + Schema: e.args[1].(string), + Table: e.args[2].(string), + ColumnType: e.args[3].([]byte), + Rows: e.args[4].([][]interface{}), + }, + } + var eventType replication.EventType + switch e.typ { + case Write: + eventType = replication.WRITE_ROWS_EVENTv2 + case Update: + eventType = replication.UPDATE_ROWS_EVENTv2 + case Delete: + eventType = replication.DELETE_ROWS_EVENTv2 + default: + c.Fatal(fmt.Sprintf("mock event generator don't support event type: %d", e.typ)) + } + evs, _, err := d.eventsGenerator.GenDMLEvents(eventType, dmlData, 0) + c.Assert(err, IsNil) + events = append(events, evs...) + } + } + return events +} + +// func (d *testDataValidatorSuite) TestMock(c *C) { +// db, mock, err := sqlmock.New() +// c.Assert(err, IsNil) +// mock.ExpectQuery(".*").WillReturnRows(sqlmock.NewRows([]string{"a", "b"})) +// rows, err := db.Query("SELECT * FROM test;") +// c.Assert(err, IsNil) +// iter := &RowDataIteratorImpl{ +// Rows: rows, +// } +// res, err := iter.Next() +// c.Assert(res, IsNil) +// c.Assert(err, IsNil) +// } + func (d *testDataValidatorSuite) TestDoValidate(c *C) { + type testCase struct { + schemaName string + tblName string + creatSQL string + binlogEvs []mockBinlogEvent + selectSQLs []string // mock in toDB + selectArgs [][]string + retRows [][][]string + colNames []string + failRowCnt int + failRowPKs []string + } + batchSize := 2 + testCases := []testCase{ + { + // test case 1: + schemaName: "test1", + tblName: "tbl1", + creatSQL: `create table if not exists test1.tbl1( + a int, + b int, + c text, + primary key(a, b));`, + binlogEvs: []mockBinlogEvent{ + {typ: Write, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(1), int32(2), "some data1"}}}}, + {typ: Write, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(3), int32(4), "some data2"}}}}, + {typ: Update, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(1), int32(2), "some data1"}, {int32(1), int32(3), "some data3"}}}}, + {typ: Delete, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(3), int32(4), "some data2"}}}}, + + {typ: Write, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(5), int32(6), "some data4"}}}}, + {typ: Update, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(5), int32(6), "some data4"}, {int32(5), int32(7), "some data4"}}}}, + }, + selectSQLs: []string{ + "SELECT .* FROM .*test1.* WHERE .*a,b.* in \\(\\(\\?,\\?\\),\\(\\?,\\?\\)\\).*", // batch1: insert row1, row2 + "SELECT .* FROM .*test1.* WHERE .*a,b.* in \\(\\(\\?,\\?\\)\\).*", // batch2: update row1 + "SELECT .* FROM .*test1.* WHERE .*a,b.* in \\(\\(\\?,\\?\\)\\).*", // batch2: delete row2 + "SELECT .* FROM .*test1.* WHERE .*a,b.* in \\(\\(\\?,\\?\\),\\(\\?,\\?\\)\\).*", // batch3: insert and update row3. pk changed and probably caused false negative + }, + selectArgs: [][]string{ + {"1", "2", "3", "4"}, + {"1", "3"}, + {"3", "4"}, + {"5", "6", "5", "7"}, + }, + retRows: [][][]string{ + { + {"1", "2", "some data1"}, // insert + {"3", "4", "some data2"}, + }, + { + {"1", "3", "some data3"}, // update + }, + { + {}, // delete + }, + { + {"5", "7", "some data4"}, + }, + }, + colNames: []string{"a", "b", "c"}, + failRowCnt: 1, // one false negative + failRowPKs: []string{"5-6"}, + }, + } + for _, testCase := range testCases { + var ( + fromDB, toDB *sql.DB + fromMock, toMock sqlmock.Sqlmock + err error + ) + fromDB, fromMock, err = sqlmock.New() + c.Assert(err, IsNil) + toDB, toMock, err = sqlmock.New() + c.Assert(err, IsNil) + syncerObj := NewSyncer(d.cfg, nil, nil) + c.Assert(syncerObj, NotNil) + validator := NewContinuousDataValidator(d.cfg, syncerObj) + validator.Start(pb.Stage_Paused) // init but will return soon + validator.result = pb.ProcessResult{} // clear error + validator.workerCnt = 1 + validator.retryInterval = 100 // never retry + events1 := testCase.binlogEvs + mockStreamerProducer := &MockStreamProducer{d.generateEvents(events1, c)} + mockStreamer, err := mockStreamerProducer.generateStreamer(binlog.NewLocation("")) + c.Assert(err, IsNil) + validator.streamerController = &StreamerController{ + streamerProducer: mockStreamerProducer, + streamer: mockStreamer, + closed: false, + } + // validate every 2 rows updated + validator.batchRowCnt = batchSize + validator.fromDB = conn.NewBaseDB(fromDB, func() {}) + validator.fromDBConn = d.genDBConn(c, fromDB) + validator.toDB = conn.NewBaseDB(toDB, func() {}) + validator.toDBConn = d.genDBConn(c, toDB) + fromMock.ExpectQuery("SHOW CREATE TABLE " + fmt.Sprintf("`%s`.`%s`", testCase.schemaName, testCase.tblName)).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow( + "tbl1", testCase.creatSQL, + ), + ) + fromMock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows( + fromMock.NewRows([]string{"Variable_name", "Value"}).AddRow( + "sql_mode", "", + ), + ) + for i := range testCase.selectSQLs { + rawRetRows := testCase.retRows[i] + rows := sqlmock.NewRows(testCase.colNames) + for j := range rawRetRows { + if len(rawRetRows[j]) == 0 { + break // for delete query + } + rowVals := []driver.Value{} + for k := range rawRetRows[j] { + rowVals = append(rowVals, rawRetRows[j][k]) + } + rows.AddRow(rowVals...) + } + args := []driver.Value{} + for _, arg := range testCase.selectArgs[i] { + args = append(args, arg) + } + toMock.ExpectQuery(testCase.selectSQLs[i]).WithArgs(args...).WillReturnRows(rows) + } + validator.doValidate() + // wait for all routine finished + validator.cancel() + // validator.wg.Wait() + time.Sleep(5 * time.Second) + fmt.Printf("results: %v\n", validator.result.Errors) + // failed row + c.Assert(int(validator.failedRowCnt.Load()), Equals, testCase.failRowCnt) + // valid table + fullTableName := testCase.schemaName + "." + testCase.tblName + table, ok := validator.diffTables[fullTableName] + c.Assert(ok, IsTrue) + c.Assert(len(table.Info.Columns), Equals, len(testCase.colNames)) + if testCase.failRowCnt > 0 { + // validate failed rows + _, ok := validator.failedChangesMap[0][fullTableName] + c.Assert(ok, IsTrue) + allRowsPKs := []string{} + for key := range validator.failedChangesMap[0][fullTableName].rows { + allRowsPKs = append(allRowsPKs, key) + } + sort.Slice(allRowsPKs, func(l, r int) bool { + return allRowsPKs[l] < allRowsPKs[r] + }) + c.Assert(allRowsPKs, DeepEquals, testCase.failRowPKs) + } + c.Assert(len(validator.result.Errors), Equals, 1) + } } From f6c6906f2908e50f4fb7ab38b04b651a9489d42a Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Mon, 14 Feb 2022 15:26:04 +0800 Subject: [PATCH 05/36] feat: add test case and fix bugs --- dm/syncer/data_validator.go | 74 +++++++++++----------- dm/syncer/data_validator_test.go | 102 +++++++++++++++++++------------ 2 files changed, 102 insertions(+), 74 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index bbdcf1817a8..da1d673bbf5 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -320,7 +320,7 @@ func (v *DataValidator) doValidate() { e, err := v.streamerController.GetEvent(tctx) if err != nil { // todo: validation paused? - v.fillResult(errors.New("getting binlog event failed"), true) + v.fillResult(terror.Annotate(err, "fail to get binlog from stream controller"), true) return } // todo: configuring the time or use checkpoint @@ -381,7 +381,7 @@ func (v *DataValidator) rowsEventProcessRoutine() { for { select { case <-v.ctx.Done(): - fmt.Printf("rowsEventProcessRoutine Done\n") + v.L.Debug("validator row event process unit done") v.wg.Done() return case e := <-v.rowsEventChan: @@ -412,7 +412,7 @@ func (v *DataValidator) workerValidateTableChange(workerID int) { v.pendingRowCnt.Sub(int64(v.getRowCount(change))) // atomic value v.failedRowCnt.Add(int64(deltaFailed)) // atomic value case <-v.ctx.Done(): - fmt.Printf("workerValidateTableChange %d Done\n", workerID) + v.L.Debug("validator worker done", zap.Int("workerID", workerID)) v.wg.Done() return case <-time.After(v.retryInterval): @@ -435,7 +435,7 @@ func (v *DataValidator) validateTaskDispatchRoutine() { case change := <-v.pendingChangeCh: v.dispatchRowChange(change) // dispatch change to worker case <-v.ctx.Done(): - fmt.Printf("validateTaskDispatchRoutine Done\n") + v.L.Debug("validator task dispatch done") v.wg.Done() return } @@ -521,16 +521,14 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re } table = v.diffTables[fullTableName] } - if table == nil { - return nil - } if table.PrimaryKey == nil { - errMsg := fmt.Sprintf("no primary key for %s.%s", table.Schema, table.Table) - return errors.New(errMsg) + err := errors.Errorf("missing primary key for table `%s`.`%s`", schemaName, tableName) + return err } for _, cols := range ev.SkippedColumns { if len(cols) > 0 { - return errors.New("") + err := errors.Errorf("unexpected skipped columns for table `%s`.`%s`", schemaName, tableName) + return err } } changeType := getRowChangeType(header.EventType) @@ -540,10 +538,6 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re } v.changeEventCount[changeType]++ - init, step := 0, 1 - if changeType == rowUpdated { - init, step = 1, 2 - } pk := table.PrimaryKey pkIndices := make([]int, len(pk.Columns)) for i, col := range pk.Columns { @@ -552,7 +546,7 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re rowCount := v.getRowCount(v.accumulatedChanges) change := v.accumulatedChanges[fullTableName] - for i := init; i < len(ev.Rows); i += step { + for i := 0; i < len(ev.Rows); i++ { row := ev.Rows[i] pkValue := make([]string, len(pk.Columns)) for _, idx := range pkIndices { @@ -569,16 +563,29 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re } key := strings.Join(pkValue, "-") val, ok := change.rows[key] - if !ok { - // this row hasn't been changed before + if !ok && (changeType != rowUpdated || i%2 != 0) { + // insert a new row change iff: + // 1. not an update binlog record (e.g. insert/delete) OR + // 2. the new value in the update binlog val = &rowChange{pk: pkValue} change.rows[key] = val rowCount++ v.pendingRowCnt.Inc() + } else if ok && changeType == rowUpdated && i%2 == 0 { + // if this row change exists and updated now: + // mark the `old_row_val` as deleted + // delete it and replace this record with the `new_row_val` later in the next iteration + val.theType = rowDeleted + continue + } + if val != nil { + val.data = row + val.theType = changeType + val.lastMeetTs = int64(header.Timestamp) + } else { + // row change NOT exists and this row change record is an update binlog with `old_row_val` + continue } - val.data = row - val.theType = changeType - val.lastMeetTs = int64(header.Timestamp) if rowCount >= v.batchRowCnt { v.pendingChangeCh <- v.accumulatedChanges @@ -667,16 +674,13 @@ func (v *DataValidator) validateChanges(table *TableDiff, rows []*rowChange, del failedRows, err = v.validateInsertAndUpdateRows(rows, cond) } if err != nil { - v.fillResult(err, false) + v.L.Warn("fail to validate row changes of table", zap.Error(err)) return [][]string{} } return failedRows } // remove previous failed rows related to current batch of rows -// e.g. Assuming that one row was modified twice and successfully migrated to downstream. -// the validator might get false negative result when validating the first update binlog record -// but it must finally get true positive after validating the second update record. // This function updates the failed rows every time after validating func (v *DataValidator) updateFailedChangesByWorker(all, failed map[string]*tableChange, workerID int) int { failedChanges := v.failedChangesMap[workerID] @@ -689,8 +693,10 @@ func (v *DataValidator) updateFailedChangesByWorker(all, failed map[string]*tabl } for _, r := range val.rows { key := strings.Join(r.pk, "-") - delete(prevFailed.rows, key) - deltaFailed-- + if _, ok := prevFailed.rows[key]; ok { + delete(prevFailed.rows, key) + deltaFailed-- + } } } for k, val := range failed { @@ -740,14 +746,12 @@ func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Con var err error upstreamRowsIterator, err = getRowChangeIterator(cond.Table, rows) if err != nil { - return nil, errors.New("get row change iter fails") + return nil, err } defer upstreamRowsIterator.Close() - fmt.Printf("cond: %v\n", cond) downstreamRowsIterator, err = getRowsFrom(cond, v.toDBConn) if err != nil { - fmt.Printf("get row fails: %s\n", err.Error()) - return nil, errors.New("get rows fails") + return nil, err } defer downstreamRowsIterator.Close() @@ -793,7 +797,7 @@ func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Con eq, cmp, err := v.compareData(lastUpstreamData, lastDownstreamData, orderKeyCols, tableInfo.Columns) if err != nil { - return nil, errors.New("") + return nil, err } if eq { lastDownstreamData = nil @@ -904,10 +908,10 @@ func (v *DataValidator) compareData(map1, map2 map[string]*dbutil.ColumnData, or for _, column := range columns { if data1, ok = map1[column.Name.O]; !ok { - return false, 0, errors.Errorf("upstream don't have key %s", key) + return false, 0, errors.Errorf("upstream doesn't have key %s", key) } if data2, ok = map2[column.Name.O]; !ok { - return false, 0, errors.Errorf("downstream don't have key %s", key) + return false, 0, errors.Errorf("downstream doesn't have key %s", key) } str1 = string(data1.Data) str2 = string(data2.Data) @@ -942,11 +946,11 @@ func (v *DataValidator) compareData(map1, map2 map[string]*dbutil.ColumnData, or // Not Equal. Compare orderkeycolumns. for _, col := range orderKeyCols { if data1, ok = map1[col.Name.O]; !ok { - err = errors.Errorf("don't have key %s", col.Name.O) + err = errors.Errorf("upstream doesn't have key %s", col.Name.O) return } if data2, ok = map2[col.Name.O]; !ok { - err = errors.Errorf("don't have key %s", col.Name.O) + err = errors.Errorf("downstream doesn't have key %s", col.Name.O) return } diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index 3fbbb0aba83..332d1fc1503 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -22,6 +22,8 @@ import ( "sort" "time" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/DATA-DOG/go-sqlmock" "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" @@ -67,7 +69,7 @@ func (d *testDataValidatorSuite) SetUpSuite(c *C) { To: config.GetDBConfigForTest(), ServerID: 101, MetaSchema: "test", - Name: "syncer_ut", + Name: "validator_ut", ShadowTableRules: []string{config.DefaultShadowTableRules}, TrashTableRules: []string{config.DefaultTrashTableRules}, Mode: config.ModeIncrement, @@ -79,6 +81,7 @@ func (d *testDataValidatorSuite) SetUpSuite(c *C) { d.cfg.To.Adjust() d.cfg.UseRelay = false + c.Assert(log.InitLogger(&log.Config{}), IsNil) } func (d *testDataValidatorSuite) genDBConn(c *C, db *sql.DB) *dbconn.DBConn { @@ -517,20 +520,6 @@ func (d *testDataValidatorSuite) generateEvents(binlogEvents mockBinlogEvents, c return events } -// func (d *testDataValidatorSuite) TestMock(c *C) { -// db, mock, err := sqlmock.New() -// c.Assert(err, IsNil) -// mock.ExpectQuery(".*").WillReturnRows(sqlmock.NewRows([]string{"a", "b"})) -// rows, err := db.Query("SELECT * FROM test;") -// c.Assert(err, IsNil) -// iter := &RowDataIteratorImpl{ -// Rows: rows, -// } -// res, err := iter.Next() -// c.Assert(res, IsNil) -// c.Assert(err, IsNil) -// } - func (d *testDataValidatorSuite) TestDoValidate(c *C) { type testCase struct { schemaName string @@ -538,7 +527,6 @@ func (d *testDataValidatorSuite) TestDoValidate(c *C) { creatSQL string binlogEvs []mockBinlogEvent selectSQLs []string // mock in toDB - selectArgs [][]string retRows [][][]string colNames []string failRowCnt int @@ -547,7 +535,6 @@ func (d *testDataValidatorSuite) TestDoValidate(c *C) { batchSize := 2 testCases := []testCase{ { - // test case 1: schemaName: "test1", tblName: "tbl1", creatSQL: `create table if not exists test1.tbl1( @@ -566,16 +553,11 @@ func (d *testDataValidatorSuite) TestDoValidate(c *C) { {typ: Update, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(5), int32(6), "some data4"}, {int32(5), int32(7), "some data4"}}}}, }, selectSQLs: []string{ - "SELECT .* FROM .*test1.* WHERE .*a,b.* in \\(\\(\\?,\\?\\),\\(\\?,\\?\\)\\).*", // batch1: insert row1, row2 - "SELECT .* FROM .*test1.* WHERE .*a,b.* in \\(\\(\\?,\\?\\)\\).*", // batch2: update row1 - "SELECT .* FROM .*test1.* WHERE .*a,b.* in \\(\\(\\?,\\?\\)\\).*", // batch2: delete row2 - "SELECT .* FROM .*test1.* WHERE .*a,b.* in \\(\\(\\?,\\?\\),\\(\\?,\\?\\)\\).*", // batch3: insert and update row3. pk changed and probably caused false negative - }, - selectArgs: [][]string{ - {"1", "2", "3", "4"}, - {"1", "3"}, - {"3", "4"}, - {"5", "6", "5", "7"}, + "SELECT .* FROM .*test1.* WHERE .*a,b.* in .*", // batch1: insert row1, row2 + "SELECT .* FROM .*test1.* WHERE .*a,b.* in .*", // batch2: update row1 + "SELECT .* FROM .*test1.* WHERE .*a,b.* in .*", // batch2: delete row2 + "SELECT .* FROM .*test1.* WHERE .*a,b.* in .*", // batch3: insert and update row3, only query (5, 7) + "SELECT .* FROM .*test1.* WHERE .*a,b.* in .*", // batch3: update query (5, 6) }, retRows: [][][]string{ { @@ -591,10 +573,57 @@ func (d *testDataValidatorSuite) TestDoValidate(c *C) { { {"5", "7", "some data4"}, }, + { + {}, + }, + }, + colNames: []string{"a", "b", "c"}, + failRowCnt: 0, + failRowPKs: []string{}, + }, + { + // stale read in downstream and got erronous result + // but row2's primary key is reused and inserted again + // the error is restored + schemaName: "test2", + tblName: "tbl2", + creatSQL: `create table if not exists test2.tbl2( + a varchar(10), + b int, + c float, + primary key(a));`, + binlogEvs: []mockBinlogEvent{ + {typ: Write, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val1", int32(1), float32(1.2)}}}}, + {typ: Write, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val2", int32(2), float32(2.2)}}}}, + + {typ: Delete, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val1", int32(1), float32(1.2)}}}}, + {typ: Delete, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val2", int32(2), float32(2.2)}}}}, + + {typ: Write, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val3", int32(3), float32(3.2)}}}}, + {typ: Write, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val2", int32(2), float32(4.2)}}}}, + }, + selectSQLs: []string{ + "SELECT .* FROM .*test2.* WHERE .*a.* in.*", // batch1: query row1 and row2 + "SELECT .* FROM .*test2.* WHERE .*a.* in.*", // batch2: query row1 and row2 + "SELECT .* FROM .*test2.* WHERE .*a.* in.*", // batch3: query row2 and row3 + }, + retRows: [][][]string{ + { + {"val1", "1", "1.2"}, + {"val2", "2", "2.2"}, + }, + { + {"val1", "1", "1.2"}, + {"val2", "2", "2.2"}, + }, + { + {"val2", "2", "4.2"}, + {"val3", "3", "3.2"}, + }, }, colNames: []string{"a", "b", "c"}, - failRowCnt: 1, // one false negative - failRowPKs: []string{"5-6"}, + failRowCnt: 1, + failRowPKs: []string{"val1"}, }, } for _, testCase := range testCases { @@ -613,7 +642,7 @@ func (d *testDataValidatorSuite) TestDoValidate(c *C) { validator.Start(pb.Stage_Paused) // init but will return soon validator.result = pb.ProcessResult{} // clear error validator.workerCnt = 1 - validator.retryInterval = 100 // never retry + validator.retryInterval = 100 * time.Second // never retry events1 := testCase.binlogEvs mockStreamerProducer := &MockStreamProducer{d.generateEvents(events1, c)} mockStreamer, err := mockStreamerProducer.generateStreamer(binlog.NewLocation("")) @@ -652,18 +681,13 @@ func (d *testDataValidatorSuite) TestDoValidate(c *C) { } rows.AddRow(rowVals...) } - args := []driver.Value{} - for _, arg := range testCase.selectArgs[i] { - args = append(args, arg) - } - toMock.ExpectQuery(testCase.selectSQLs[i]).WithArgs(args...).WillReturnRows(rows) + toMock.ExpectQuery(testCase.selectSQLs[i]).WillReturnRows(rows) } validator.doValidate() // wait for all routine finished + time.Sleep(1 * time.Second) validator.cancel() - // validator.wg.Wait() - time.Sleep(5 * time.Second) - fmt.Printf("results: %v\n", validator.result.Errors) + validator.wg.Wait() // failed row c.Assert(int(validator.failedRowCnt.Load()), Equals, testCase.failRowCnt) // valid table @@ -684,6 +708,6 @@ func (d *testDataValidatorSuite) TestDoValidate(c *C) { }) c.Assert(allRowsPKs, DeepEquals, testCase.failRowPKs) } - c.Assert(len(validator.result.Errors), Equals, 1) + c.Assert(len(validator.result.Errors), Equals, 0) } } From e718a3b2a76324f1cfe34e6352d7d0659854aef3 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Mon, 14 Feb 2022 15:04:09 +0800 Subject: [PATCH 06/36] rename, close db on err, move some init to ctor --- dm/syncer/data_validator.go | 89 ++++++++++++++++---------------- dm/syncer/data_validator_test.go | 2 +- dm/syncer/validator_cond.go | 2 +- dm/syncer/validator_cond_test.go | 6 +-- 4 files changed, 50 insertions(+), 49 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index da1d673bbf5..7d84c3ece2e 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -16,18 +16,17 @@ package syncer import ( "context" "database/sql" - "hash/fnv" - "sort" - - "github.com/pingcap/errors" - "fmt" + "hash/fnv" "math" + "sort" "strconv" "strings" "sync" "time" + "github.com/pingcap/errors" + "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" tidbmysql "github.com/pingcap/tidb/parser/mysql" @@ -36,6 +35,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/tiflow/dm/dm/unit" @@ -57,18 +57,10 @@ const ( type rowChangeType int -// table info -type TableDiff struct { - // Schema represents the database name. - Schema string `json:"schema"` - - // Table represents the table name. - Table string `json:"table"` - - // Info is the parser.TableInfo, include some meta infos for this table. - // It used for TiDB/MySQL/MySQL Shard sources. - Info *model.TableInfo `json:"info"` - +type validateTableInfo struct { + Schema string + Name string + Info *model.TableInfo PrimaryKey *model.IndexInfo ColumnMap map[string]*model.ColumnInfo } @@ -125,7 +117,7 @@ const ( // binlog changes are clustered into table changes // the validator validates changes of table-grain at a time type tableChange struct { - table *TableDiff + table *validateTableInfo rows map[string]*rowChange } @@ -175,7 +167,7 @@ type DataValidator struct { pendingChangeCh chan map[string]*tableChange changeEventCount []int validationTimer *time.Timer - diffTables map[string]*TableDiff + tables map[string]*validateTableInfo workerCnt int pendingChangeChs map[int]chan map[string]*tableChange // replace pendingChangeCh failedChangesMap map[int]map[string]*tableChange // replace failedChanges @@ -188,6 +180,23 @@ func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *D stage: pb.Stage_Stopped, } c.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) + + c.workerCnt = defaultWorkerCnt + c.pendingChangeChs = make(map[int]chan map[string]*tableChange) + c.failedChangesMap = make(map[int]map[string]*tableChange) + for i := 0; i < c.workerCnt; i++ { + c.pendingChangeChs[i] = make(chan map[string]*tableChange) + c.failedChangesMap[i] = make(map[string]*tableChange) + } + c.batchRowCnt = batchRowCount + c.validationTimer = time.NewTimer(validationInterval) + c.rowsEventChan = make(chan *replication.BinlogEvent) + c.pendingChangeCh = make(chan map[string]*tableChange) + c.tables = make(map[string]*validateTableInfo) + c.changeEventCount = make([]int, 4) + c.accumulatedChanges = make(map[string]*tableChange) + c.retryInterval = retryInterval + return c } @@ -198,27 +207,17 @@ func (v *DataValidator) initialize() error { var err error defer func() { - if err != nil && v.fromDB != nil { + if err == nil { + return + } + if v.fromDB != nil { v.fromDB.Close() } + if v.toDB != nil { + v.toDB.Close() + } }() - v.workerCnt = defaultWorkerCnt - v.pendingChangeChs = make(map[int]chan map[string]*tableChange) - v.failedChangesMap = make(map[int]map[string]*tableChange) - for i := 0; i < v.workerCnt; i++ { - v.pendingChangeChs[i] = make(chan map[string]*tableChange) - v.failedChangesMap[i] = make(map[string]*tableChange) - } - v.batchRowCnt = batchRowCount - v.validationTimer = time.NewTimer(validationInterval) - v.rowsEventChan = make(chan *replication.BinlogEvent) - v.pendingChangeCh = make(chan map[string]*tableChange) - v.diffTables = make(map[string]*TableDiff) - v.changeEventCount = make([]int, 4) - v.accumulatedChanges = make(map[string]*tableChange) - v.retryInterval = retryInterval - dbCfg := v.cfg.From dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxDMLConnectionTimeout) var fromDBConns, toDBConns []*dbconn.DBConn @@ -227,11 +226,13 @@ func (v *DataValidator) initialize() error { return err } v.fromDBConn = fromDBConns[0] + v.toDB, toDBConns, err = dbconn.CreateConns(tctx, v.cfg, &dbCfg, 1) if err != nil { return err } v.toDBConn = toDBConns[0] + v.timezone, err = str2TimezoneOrFromDB(tctx, v.cfg.Timezone, &v.cfg.To) if err != nil { return err @@ -481,10 +482,10 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re schemaName, tableName := string(ev.Table.Schema), string(ev.Table.Table) fullTableName := fmt.Sprintf("%s.%s", schemaName, tableName) var ( - table *TableDiff + table *validateTableInfo ok bool ) - if table, ok = v.diffTables[fullTableName]; !ok { + if table, ok = v.tables[fullTableName]; !ok { // table not found in cache // try getting table info from the upstream tctx := tcontext.NewContext(v.ctx, v.L) @@ -512,14 +513,14 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re primaryIdx = idx } } - v.diffTables[fullTableName] = &TableDiff{ + v.tables[fullTableName] = &validateTableInfo{ Schema: schemaName, - Table: tableName, + Name: tableName, Info: tableInfo, PrimaryKey: primaryIdx, ColumnMap: columnMap, } - table = v.diffTables[fullTableName] + table = v.tables[fullTableName] } if table.PrimaryKey == nil { err := errors.Errorf("missing primary key for table `%s`.`%s`", schemaName, tableName) @@ -660,7 +661,7 @@ func (v *DataValidator) validateTableChange(tableChanges map[string]*tableChange return failedChanges } -func (v *DataValidator) validateChanges(table *TableDiff, rows []*rowChange, deleteChange bool) [][]string { +func (v *DataValidator) validateChanges(table *validateTableInfo, rows []*rowChange, deleteChange bool) [][]string { pkValues := make([][]string, 0, len(rows)) for _, r := range rows { pkValues = append(pkValues, r.pk) @@ -826,7 +827,7 @@ func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Con func getRowsFrom(cond *Cond, conn *dbconn.DBConn) (RowDataIterator, error) { tctx := tcontext.NewContext(context.Background(), log.L()) - fullTableName := dbutil.TableName(cond.Table.Schema, cond.Table.Table) + fullTableName := dbutil.TableName(cond.Table.Schema, cond.Table.Name) orderKeys, _ := dbutil.SelectUniqueOrderKey(cond.Table.Info) columnNames := make([]string, 0, len(cond.Table.Info.Columns)) for _, col := range cond.Table.ColumnMap { @@ -845,7 +846,7 @@ func getRowsFrom(cond *Cond, conn *dbconn.DBConn) (RowDataIterator, error) { return newRowDataIter, nil } -func getRowChangeIterator(table *TableDiff, rows []*rowChange) (RowDataIterator, error) { +func getRowChangeIterator(table *validateTableInfo, rows []*rowChange) (RowDataIterator, error) { sort.Slice(rows, func(i, j int) bool { left, right := rows[i], rows[j] for idx := range left.pk { diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index 332d1fc1503..d122e6cff33 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -692,7 +692,7 @@ func (d *testDataValidatorSuite) TestDoValidate(c *C) { c.Assert(int(validator.failedRowCnt.Load()), Equals, testCase.failRowCnt) // valid table fullTableName := testCase.schemaName + "." + testCase.tblName - table, ok := validator.diffTables[fullTableName] + table, ok := validator.tables[fullTableName] c.Assert(ok, IsTrue) c.Assert(len(table.Info.Columns), Equals, len(testCase.colNames)) if testCase.failRowCnt > 0 { diff --git a/dm/syncer/validator_cond.go b/dm/syncer/validator_cond.go index 3c6b5865162..4ff2ac1e513 100644 --- a/dm/syncer/validator_cond.go +++ b/dm/syncer/validator_cond.go @@ -18,7 +18,7 @@ import ( ) type Cond struct { - Table *TableDiff + Table *validateTableInfo PkValues [][]string } diff --git a/dm/syncer/validator_cond_test.go b/dm/syncer/validator_cond_test.go index 88a8e8bf0f2..4361b59670d 100644 --- a/dm/syncer/validator_cond_test.go +++ b/dm/syncer/validator_cond_test.go @@ -29,7 +29,7 @@ type testCondSuite struct{} var _ = Suite(&testCondSuite{}) -func getTableDiff(c *C, db *sql.DB, schemaName, tableName, creatSQL string) *TableDiff { +func getTableDiff(c *C, db *sql.DB, schemaName, tableName, creatSQL string) *validateTableInfo { var ( err error parser2 *parser.Parser @@ -49,9 +49,9 @@ func getTableDiff(c *C, db *sql.DB, schemaName, tableName, creatSQL string) *Tab primaryIdx = idx } } - tableDiff := &TableDiff{ + tableDiff := &validateTableInfo{ Schema: schemaName, - Table: tableName, + Name: tableName, Info: tableInfo, PrimaryKey: primaryIdx, ColumnMap: columnMap, From 95156d95de31d59376e4929a168a2d806bb31534 Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Mon, 14 Feb 2022 17:17:35 +0800 Subject: [PATCH 07/36] fix: conflict import --- dm/syncer/data_validator.go | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 7d84c3ece2e..b5737059443 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -25,17 +25,15 @@ import ( "sync" "time" - "github.com/pingcap/errors" - "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" + "github.com/pingcap/errors" + "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb/parser/model" tidbmysql "github.com/pingcap/tidb/parser/mysql" "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/tidb-tools/pkg/dbutil" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/tiflow/dm/dm/unit" From c330483e8807fdaaa9348a7fbc4614890680f2e2 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Mon, 14 Feb 2022 19:54:25 +0800 Subject: [PATCH 08/36] fix --- dm/syncer/data_validator.go | 19 +++++++++++++------ dm/syncer/validator_cond.go | 5 ++--- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index b5737059443..3d8e8e70dd8 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -300,12 +300,21 @@ func (v *DataValidator) fillResult(err error, needLock bool) { } } +func (v *DataValidator) processError(err error) { + v.fillResult(err, true) + + if errors.Cause(err) != context.Canceled { + v.Stop() + } +} + +// doValidate: runs in a separate goroutine func (v *DataValidator) doValidate() { tctx := tcontext.NewContext(v.ctx, v.L) if v.streamerController.IsClosed() { err := v.streamerController.Start(tctx, lastLocation) if err != nil { - v.fillResult(terror.Annotate(err, "fail to restart streamer controller"), true) + v.processError(terror.Annotate(err, "fail to start streamer controller")) return } } @@ -314,12 +323,12 @@ func (v *DataValidator) doValidate() { v.wg.Add(2) go v.rowsEventProcessRoutine() go v.validateTaskDispatchRoutine() + var latestPos mysql.Position for { e, err := v.streamerController.GetEvent(tctx) if err != nil { - // todo: validation paused? - v.fillResult(terror.Annotate(err, "fail to get binlog from stream controller"), true) + v.processError(terror.Annotate(err, "fail to get binlog from stream controller")) return } // todo: configuring the time or use checkpoint @@ -353,13 +362,11 @@ func (v *DataValidator) Stop() { return } + v.cancel() v.streamerController.Close() v.fromDB.Close() v.toDB.Close() - if v.cancel != nil { - v.cancel() - } v.wg.Wait() v.stage = pb.Stage_Stopped } diff --git a/dm/syncer/validator_cond.go b/dm/syncer/validator_cond.go index 4ff2ac1e513..128b1134d79 100644 --- a/dm/syncer/validator_cond.go +++ b/dm/syncer/validator_cond.go @@ -46,10 +46,9 @@ func (c *Cond) GetWhere() string { b.WriteString(pk.Columns[i].Name.O) } if !isOneKey { - b.WriteString(") in (") - } else { - b.WriteString(" in (") + b.WriteString(")") } + b.WriteString(" in (") for i := range c.PkValues { if i != 0 { b.WriteString(",") From 14b5252f3d71ab8b1f5dba41e815326a9dd669f5 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Mon, 14 Feb 2022 20:47:31 +0800 Subject: [PATCH 09/36] validation --- dm/syncer/data_validator.go | 86 +++++++++++++++++++++++++------------ 1 file changed, 59 insertions(+), 27 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 3d8e8e70dd8..ed69c33cde7 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/binlog" "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" @@ -140,10 +141,12 @@ type DataValidator struct { cfg *config.SubTaskConfig syncer *Syncer - stage pb.Stage - wg sync.WaitGroup - ctx context.Context - cancel context.CancelFunc + stage pb.Stage + wg sync.WaitGroup + errProcessWg sync.WaitGroup + errChan chan error + ctx context.Context + cancel context.CancelFunc L log.Logger fromDB *conn.BaseDB @@ -173,9 +176,10 @@ type DataValidator struct { func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *DataValidator { c := &DataValidator{ - cfg: cfg, - syncer: syncerObj, - stage: pb.Stage_Stopped, + cfg: cfg, + syncer: syncerObj, + stage: pb.Stage_Stopped, + errChan: make(chan error, 1), } c.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) @@ -271,6 +275,9 @@ func (v *DataValidator) Start(expect pb.Stage) { v.doValidate() }() + v.errProcessWg.Add(1) + go v.errorProcessRoutine() + v.stage = pb.Stage_Running } @@ -300,21 +307,36 @@ func (v *DataValidator) fillResult(err error, needLock bool) { } } -func (v *DataValidator) processError(err error) { - v.fillResult(err, true) +func (v *DataValidator) errorProcessRoutine() { + v.errProcessWg.Done() + for { + select { + case err := <-v.errChan: + v.fillResult(err, true) - if errors.Cause(err) != context.Canceled { - v.Stop() + if errors.Cause(err) != context.Canceled { + v.stopInner() + } + case <-v.ctx.Done(): + return + } } } +func (v *DataValidator) waitSyncerSynced(loc *binlog.Location, event *replication.BinlogEvent) error { + // TODO + return nil +} + // doValidate: runs in a separate goroutine func (v *DataValidator) doValidate() { + // todo: wait syncer started, get syncer checkpoint + var location binlog.Location tctx := tcontext.NewContext(v.ctx, v.L) if v.streamerController.IsClosed() { - err := v.streamerController.Start(tctx, lastLocation) + err := v.streamerController.Start(tctx, location) if err != nil { - v.processError(terror.Annotate(err, "fail to start streamer controller")) + v.errChan <- terror.Annotate(err, "fail to start streamer controller") return } } @@ -324,37 +346,46 @@ func (v *DataValidator) doValidate() { go v.rowsEventProcessRoutine() go v.validateTaskDispatchRoutine() - var latestPos mysql.Position + var currLoc binlog.Location for { e, err := v.streamerController.GetEvent(tctx) if err != nil { - v.processError(terror.Annotate(err, "fail to get binlog from stream controller")) + v.errChan <- terror.Annotate(err, "fail to get binlog from stream controller") return } - // todo: configuring the time or use checkpoint - eventTime := time.Unix(int64(e.Header.Timestamp), 0) - lag := time.Since(eventTime) - if lag < defaultDelay { - time.Sleep(defaultDelay - lag) + if utils.IsFakeRotateEvent(e.Header) { + continue + } + + // wait until syncer synced that event + if err := v.waitSyncerSynced(&currLoc, e); err != nil { + v.errChan <- terror.Annotate(err, "failed to wait syncer") + return } switch ev := e.Event.(type) { case *replication.RotateEvent: - latestPos.Name = string(ev.NextLogName) + currLoc.Position = mysql.Position{Name: string(ev.NextLogName), Pos: binlog.FileHeaderLen} case *replication.QueryEvent: // TODO not processed now + currLoc.Position.Pos = e.Header.LogPos case *replication.RowsEvent: + currLoc.Position.Pos = e.Header.LogPos select { case v.rowsEventChan <- e: case <-v.ctx.Done(): return } } - latestPos.Pos = e.Header.LogPos } } func (v *DataValidator) Stop() { + v.stopInner() + v.errProcessWg.Wait() +} + +func (v *DataValidator) stopInner() { v.Lock() defer v.Unlock() if v.stage != pb.Stage_Running { @@ -384,15 +415,16 @@ func (v *DataValidator) Stage() pb.Stage { } func (v *DataValidator) rowsEventProcessRoutine() { + defer v.wg.Done() for { select { case <-v.ctx.Done(): v.L.Debug("validator row event process unit done") - v.wg.Done() return case e := <-v.rowsEventChan: if err := v.processEventRows(e.Header, e.Event.(*replication.RowsEvent)); err != nil { v.L.Warn("failed to process event: ", zap.Reflect("error", err)) + return } case <-v.validationTimer.C: rowCount := v.getRowCount(v.accumulatedChanges) @@ -405,7 +437,8 @@ func (v *DataValidator) rowsEventProcessRoutine() { } } -func (v *DataValidator) workerValidateTableChange(workerID int) { +func (v *DataValidator) validateTableChangeWorkerRoutine(workerID int) { + defer v.wg.Done() for { select { case change := <-v.pendingChangeChs[workerID]: @@ -419,7 +452,6 @@ func (v *DataValidator) workerValidateTableChange(workerID int) { v.failedRowCnt.Add(int64(deltaFailed)) // atomic value case <-v.ctx.Done(): v.L.Debug("validator worker done", zap.Int("workerID", workerID)) - v.wg.Done() return case <-time.After(v.retryInterval): retryChange := v.failedChangesMap[workerID] @@ -431,10 +463,11 @@ func (v *DataValidator) workerValidateTableChange(workerID int) { } func (v *DataValidator) validateTaskDispatchRoutine() { + defer v.wg.Done() // start workers here v.wg.Add(v.workerCnt) for i := 0; i < v.workerCnt; i++ { - go v.workerValidateTableChange(i) + go v.validateTableChangeWorkerRoutine(i) } for { select { @@ -442,7 +475,6 @@ func (v *DataValidator) validateTaskDispatchRoutine() { v.dispatchRowChange(change) // dispatch change to worker case <-v.ctx.Done(): v.L.Debug("validator task dispatch done") - v.wg.Done() return } } From c019bee07efb1f17b12e1536664dd676aeb133fc Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Tue, 15 Feb 2022 20:31:52 +0800 Subject: [PATCH 10/36] use info from syncer:ddl, loc --- dm/syncer/data_validator.go | 263 ++++++++++++++++++++++-------------- dm/syncer/syncer.go | 19 ++- 2 files changed, 175 insertions(+), 107 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index ed69c33cde7..6663dfaa3be 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -25,10 +25,10 @@ import ( "sync" "time" - "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser/model" tidbmysql "github.com/pingcap/tidb/parser/mysql" "go.uber.org/atomic" @@ -41,27 +41,27 @@ import ( "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/syncer/dbconn" ) const ( - defaultDelay = 5 * time.Second - retryInterval = 5 * time.Second - validationInterval = time.Second - batchRowCount = 200 - defaultWorkerCnt = 5 + checkInterval = 5 * time.Second + retryInterval = 5 * time.Second + validationInterval = 10 * time.Second + defaultBatchRowCount = 500 + defaultWorkerCnt = 5 ) -type rowChangeType int - type validateTableInfo struct { Schema string Name string Info *model.TableInfo PrimaryKey *model.IndexInfo ColumnMap map[string]*model.ColumnInfo + Target *filter.Table // target table after route } type RowDataIterator interface { @@ -105,6 +105,8 @@ func (b *RowChangeIteratorImpl) Close() { // skip: nothing to do } +type rowChangeType int + const ( rowInvalidChange rowChangeType = iota rowInsert @@ -126,6 +128,7 @@ type rowChange struct { data []interface{} theType rowChangeType lastMeetTs int64 // the last meet timestamp(in seconds) + retryCnt int // retry count } // DataValidator @@ -172,6 +175,9 @@ type DataValidator struct { workerCnt int pendingChangeChs map[int]chan map[string]*tableChange // replace pendingChangeCh failedChangesMap map[int]map[string]*tableChange // replace failedChanges + + // such as table without primary key + unsupportedTable map[string]string } func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *DataValidator { @@ -190,7 +196,7 @@ func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *D c.pendingChangeChs[i] = make(chan map[string]*tableChange) c.failedChangesMap[i] = make(map[string]*tableChange) } - c.batchRowCnt = batchRowCount + c.batchRowCnt = defaultBatchRowCount c.validationTimer = time.NewTimer(validationInterval) c.rowsEventChan = make(chan *replication.BinlogEvent) c.pendingChangeCh = make(chan map[string]*tableChange) @@ -315,6 +321,7 @@ func (v *DataValidator) errorProcessRoutine() { v.fillResult(err, true) if errors.Cause(err) != context.Canceled { + // todo: need a better way to handle err(auto resuming on some error, etc.) v.stopInner() } case <-v.ctx.Done(): @@ -328,17 +335,33 @@ func (v *DataValidator) waitSyncerSynced(loc *binlog.Location, event *replicatio return nil } +func (v *DataValidator) waitSyncerRunning() error { + for { + select { + case <-v.ctx.Done(): + return v.ctx.Err() + case <-time.After(checkInterval): + if v.syncer.IsRunning() { + return nil + } + } + } +} + // doValidate: runs in a separate goroutine func (v *DataValidator) doValidate() { - // todo: wait syncer started, get syncer checkpoint - var location binlog.Location + if err := v.waitSyncerRunning(); err != nil { + v.errChan <- terror.Annotate(err, "failed to wait syncer running") + return + } + tctx := tcontext.NewContext(v.ctx, v.L) - if v.streamerController.IsClosed() { - err := v.streamerController.Start(tctx, location) - if err != nil { - v.errChan <- terror.Annotate(err, "fail to start streamer controller") - return - } + // todo: syncer may change replication location(start from timestamp, sharding resync), how validator react? + location := v.syncer.checkpoint.GlobalPoint() + err := v.streamerController.Start(tctx, location) + if err != nil { + v.errChan <- terror.Annotate(err, "fail to start streamer controller") + return } v.L.Info("start continuous validation") @@ -353,22 +376,28 @@ func (v *DataValidator) doValidate() { v.errChan <- terror.Annotate(err, "fail to get binlog from stream controller") return } - if utils.IsFakeRotateEvent(e.Header) { - continue - } - // wait until syncer synced that event - if err := v.waitSyncerSynced(&currLoc, e); err != nil { - v.errChan <- terror.Annotate(err, "failed to wait syncer") - return + if !utils.IsFakeRotateEvent(e.Header) { + // wait until syncer synced that event + err := v.waitSyncerSynced(&currLoc, e) + if err != nil { + v.errChan <- terror.Annotate(err, "failed to wait syncer") + return + } } switch ev := e.Event.(type) { case *replication.RotateEvent: - currLoc.Position = mysql.Position{Name: string(ev.NextLogName), Pos: binlog.FileHeaderLen} + currLoc.Position.Name = string(ev.NextLogName) + currLoc.Position.Pos = uint32(ev.Position) case *replication.QueryEvent: // TODO not processed now currLoc.Position.Pos = e.Header.LogPos + err2 := currLoc.SetGTID(ev.GSet) + if err2 != nil { + v.errChan <- terror.Annotate(err2, "failed to set gtid") + return + } case *replication.RowsEvent: currLoc.Position.Pos = e.Header.LogPos select { @@ -376,6 +405,13 @@ func (v *DataValidator) doValidate() { case <-v.ctx.Done(): return } + case *replication.XIDEvent: + currLoc.Position.Pos = e.Header.LogPos + err2 := currLoc.SetGTID(ev.GSet) + if err2 != nil { + v.errChan <- terror.Annotate(err2, "failed to set gtid") + return + } } } } @@ -424,6 +460,7 @@ func (v *DataValidator) rowsEventProcessRoutine() { case e := <-v.rowsEventChan: if err := v.processEventRows(e.Header, e.Event.(*replication.RowsEvent)); err != nil { v.L.Warn("failed to process event: ", zap.Reflect("error", err)) + v.errChan <- terror.Annotate(err, "failed to process event") return } case <-v.validationTimer.C: @@ -516,64 +553,58 @@ func (v *DataValidator) dispatchRowChange(change map[string]*tableChange) { } func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *replication.RowsEvent) error { - schemaName, tableName := string(ev.Table.Schema), string(ev.Table.Table) - fullTableName := fmt.Sprintf("%s.%s", schemaName, tableName) - var ( - table *validateTableInfo - ok bool - ) - if table, ok = v.tables[fullTableName]; !ok { - // table not found in cache - // try getting table info from the upstream - tctx := tcontext.NewContext(v.ctx, v.L) - fullTableQueryName := fmt.Sprintf("`%s`.`%s`", schemaName, tableName) - createSQL, err := dbconn.GetTableCreateSQL(tctx, v.fromDBConn, fullTableQueryName) - if err != nil { - // get create table stmt failed - return err - } - parser, err := utils.GetParser(v.ctx, v.fromDB.DB) - if err != nil { - return err - } - tableInfo, err := dbutil.GetTableInfoBySQL(createSQL, parser) - if err != nil { - return err - } - columnMap := make(map[string]*model.ColumnInfo) - for _, col := range tableInfo.Columns { - columnMap[col.Name.O] = col - } - var primaryIdx *model.IndexInfo - for _, idx := range tableInfo.Indices { - if idx.Primary { - primaryIdx = idx - } + sourceTable := &filter.Table{ + Schema: string(ev.Table.Schema), + Name: string(ev.Table.Table), + } + fullTableName := sourceTable.String() + if _, ok := v.unsupportedTable[fullTableName]; ok { + return nil + } + + targetTable := v.syncer.route(sourceTable) + tableInfo, err := v.syncer.schemaTracker.GetTableInfo(sourceTable) + if err != nil { + if schema.IsTableNotExists(err) { + // not a table need to sync + return nil } - v.tables[fullTableName] = &validateTableInfo{ - Schema: schemaName, - Name: tableName, - Info: tableInfo, - PrimaryKey: primaryIdx, - ColumnMap: columnMap, + return terror.Annotate(err, "failed to get table info") + } + + columnMap := make(map[string]*model.ColumnInfo) + for _, col := range tableInfo.Columns { + columnMap[col.Name.O] = col + } + var primaryIdx *model.IndexInfo + for _, idx := range tableInfo.Indices { + if idx.Primary { + primaryIdx = idx } - table = v.tables[fullTableName] } - if table.PrimaryKey == nil { - err := errors.Errorf("missing primary key for table `%s`.`%s`", schemaName, tableName) - return err + if primaryIdx == nil { + // todo: for table without primary index, need to record in the failed table, will add it later together with checkpoint + v.unsupportedTable[fullTableName] = "without primary key" + return nil + } + + table := &validateTableInfo{ + Schema: sourceTable.Schema, + Name: sourceTable.Name, + Info: tableInfo, + PrimaryKey: primaryIdx, + ColumnMap: columnMap, + Target: targetTable, } + for _, cols := range ev.SkippedColumns { if len(cols) > 0 { - err := errors.Errorf("unexpected skipped columns for table `%s`.`%s`", schemaName, tableName) + err := errors.Errorf("unexpected skipped columns for table `%s`.`%s`", table.Schema, table.Name) return err } } + changeType := getRowChangeType(header.EventType) - if changeType == rowInvalidChange { - v.L.Info("ignoring unrecognized event", zap.Reflect("event header", header)) - return nil - } v.changeEventCount[changeType]++ pk := table.PrimaryKey @@ -584,13 +615,25 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re rowCount := v.getRowCount(v.accumulatedChanges) change := v.accumulatedChanges[fullTableName] - for i := 0; i < len(ev.Rows); i++ { - row := ev.Rows[i] - pkValue := make([]string, len(pk.Columns)) - for _, idx := range pkIndices { - pkValue[idx] = fmt.Sprintf("%v", row[idx]) + + updateRowChange := func(key string, row *rowChange) { + if val, ok := change.rows[key]; ok { + val.data = row.data + val.theType = row.theType + val.lastMeetTs = row.lastMeetTs + val.retryCnt = row.retryCnt + } else { + change.rows[key] = row + rowCount++ + v.pendingRowCnt.Inc() } + } + step := 1 + if changeType == rowUpdated { + step = 2 + } + for i := 0; i < len(ev.Rows); i += step { if change == nil { // no change of this table change = &tableChange{ @@ -599,30 +642,44 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re } v.accumulatedChanges[fullTableName] = change } - key := strings.Join(pkValue, "-") - val, ok := change.rows[key] - if !ok && (changeType != rowUpdated || i%2 != 0) { - // insert a new row change iff: - // 1. not an update binlog record (e.g. insert/delete) OR - // 2. the new value in the update binlog - val = &rowChange{pk: pkValue} - change.rows[key] = val - rowCount++ - v.pendingRowCnt.Inc() - } else if ok && changeType == rowUpdated && i%2 == 0 { - // if this row change exists and updated now: - // mark the `old_row_val` as deleted - // delete it and replace this record with the `new_row_val` later in the next iteration - val.theType = rowDeleted - continue + row := ev.Rows[i] + pkValue := make([]string, len(pk.Columns)) + for _, idx := range pkIndices { + pkValue[idx] = fmt.Sprintf("%v", row[idx]) } - if val != nil { - val.data = row - val.theType = changeType - val.lastMeetTs = int64(header.Timestamp) + key := strings.Join(pkValue, "-") + + if changeType == rowUpdated { + afterRowChangeType := changeType + afterRow := ev.Rows[i+1] + afterPkValue := make([]string, len(pk.Columns)) + for _, idx := range pkIndices { + afterPkValue[idx] = fmt.Sprintf("%v", afterRow[idx]) + } + afterKey := strings.Join(afterPkValue, "-") + if afterKey != key { + // convert to delete and insert + updateRowChange(key, &rowChange{ + pk: pkValue, + data: row, + theType: rowDeleted, + lastMeetTs: int64(header.Timestamp), + }) + afterRowChangeType = rowInsert + } + updateRowChange(afterKey, &rowChange{ + pk: afterPkValue, + data: afterRow, + theType: afterRowChangeType, + lastMeetTs: int64(header.Timestamp), + }) } else { - // row change NOT exists and this row change record is an update binlog with `old_row_val` - continue + updateRowChange(key, &rowChange{ + pk: pkValue, + data: row, + theType: changeType, + lastMeetTs: int64(header.Timestamp), + }) } if rowCount >= v.batchRowCnt { @@ -649,16 +706,16 @@ func (v *DataValidator) getRowCount(c map[string]*tableChange) int { return res } +// getRowChangeType should be called only when the event type is RowsEvent func getRowChangeType(t replication.EventType) rowChangeType { switch t { case replication.WRITE_ROWS_EVENTv0, replication.WRITE_ROWS_EVENTv1, replication.WRITE_ROWS_EVENTv2: return rowInsert case replication.UPDATE_ROWS_EVENTv0, replication.UPDATE_ROWS_EVENTv1, replication.UPDATE_ROWS_EVENTv2: return rowUpdated - case replication.DELETE_ROWS_EVENTv0, replication.DELETE_ROWS_EVENTv1, replication.DELETE_ROWS_EVENTv2: - return rowDeleted default: - return rowInvalidChange + // replication.DELETE_ROWS_EVENTv0, replication.DELETE_ROWS_EVENTv1, replication.DELETE_ROWS_EVENTv2: + return rowDeleted } } diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 96ebb22d793..e0d412f23fa 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -164,7 +164,10 @@ type Syncer struct { exprFilterGroup *ExprFilterGroup sessCtx sessionctx.Context - closed atomic.Bool + // int value of pb.Stage, stage diagram: + // new --> running --> stopped --> finished + // |-> finished + stage atomic.Int32 start atomic.Time lastTime atomic.Time @@ -248,7 +251,7 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, relay rel syncer.jobsClosed.Store(true) // not open yet syncer.waitXIDJob.Store(int64(noWait)) syncer.isTransactionEnd = true - syncer.closed.Store(false) + syncer.stage.Store(int32(pb.Stage_New)) syncer.lastBinlogSizeCount.Store(0) syncer.binlogSizeCount.Store(0) syncer.lastCount.Store(0) @@ -672,12 +675,16 @@ func (s *Syncer) Process(ctx context.Context, pr chan pb.ProcessResult) { <-newCtx.Done() // ctx or newCtx }() + s.stage.Store(int32(pb.Stage_Running)) + defer s.stage.Store(int32(pb.Stage_Stopped)) + err := s.Run(newCtx) if err != nil { // returned error rather than sent to runFatalChan // cancel goroutines created in s.Run cancel() } + s.closeJobChans() // Run returned, all jobs sent, we can close s.jobs s.wg.Wait() // wait for sync goroutine to return close(runFatalChan) // Run returned, all potential fatal sent to s.runFatalChan @@ -3374,8 +3381,12 @@ func (s *Syncer) route(table *filter.Table) *filter.Table { return &filter.Table{Schema: targetSchema, Name: targetTable} } +func (s *Syncer) IsRunning() bool { + return s.stage.Load() == int32(pb.Stage_Running) +} + func (s *Syncer) isClosed() bool { - return s.closed.Load() + return s.stage.Load() == int32(pb.Stage_Finished) } // Close closes syncer. @@ -3407,7 +3418,7 @@ func (s *Syncer) Close() { metrics.RemoveLabelValuesWithTaskInMetrics(s.cfg.Name) - s.closed.Store(true) + s.stage.Store(int32(pb.Stage_Finished)) } // stopSync stops syncing, now it used by Close and Pause From 7a058c3ef8a41147a8411d4945fa4abb0bec36b2 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Wed, 16 Feb 2022 17:02:31 +0800 Subject: [PATCH 11/36] wait until syncer synced --- dm/pkg/binlog/position.go | 4 ++ dm/syncer/data_validator.go | 80 +++++++++++++++++++++++++------------ 2 files changed, 58 insertions(+), 26 deletions(-) diff --git a/dm/pkg/binlog/position.go b/dm/pkg/binlog/position.go index ad7cfb1e719..4989606f894 100644 --- a/dm/pkg/binlog/position.go +++ b/dm/pkg/binlog/position.go @@ -428,3 +428,7 @@ func (l *Location) SetGTID(gset gmysql.GTIDSet) error { func (l *Location) GetGTID() gtid.Set { return l.gtidSet } + +func (l *Location) Update(gtidStr string) error { + return l.gtidSet.Update(gtidStr) +} diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 6663dfaa3be..b4bce7d0e44 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/tiflow/dm/dm/unit" "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" @@ -178,6 +179,7 @@ type DataValidator struct { // such as table without primary key unsupportedTable map[string]string + waitSyncerTimer *time.Timer } func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *DataValidator { @@ -205,6 +207,8 @@ func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *D c.accumulatedChanges = make(map[string]*tableChange) c.retryInterval = retryInterval + c.waitSyncerTimer = utils.NewStoppedTimer() + return c } @@ -330,9 +334,39 @@ func (v *DataValidator) errorProcessRoutine() { } } -func (v *DataValidator) waitSyncerSynced(loc *binlog.Location, event *replication.BinlogEvent) error { - // TODO - return nil +func (v *DataValidator) waitSyncerSynced(currLoc binlog.Location) error { + syncLoc := v.syncer.checkpoint.FlushedGlobalPoint() + cmp := binlog.CompareLocation(currLoc, syncLoc, v.cfg.EnableGTID) + if cmp <= 0 { + return nil + } + + fired := false + v.waitSyncerTimer.Reset(checkInterval) + defer func() { + if !fired { + if !v.waitSyncerTimer.Stop() { + <-v.waitSyncerTimer.C + } + } + }() + + for { + select { + case <-v.ctx.Done(): + return v.ctx.Err() + case <-v.waitSyncerTimer.C: + fired = true + syncLoc = v.syncer.checkpoint.FlushedGlobalPoint() + cmp = binlog.CompareLocation(currLoc, syncLoc, v.cfg.EnableGTID) + if cmp <= 0 { + return nil + } else { + v.waitSyncerTimer.Reset(checkInterval) + fired = false + } + } + } } func (v *DataValidator) waitSyncerRunning() error { @@ -357,7 +391,7 @@ func (v *DataValidator) doValidate() { tctx := tcontext.NewContext(v.ctx, v.L) // todo: syncer may change replication location(start from timestamp, sharding resync), how validator react? - location := v.syncer.checkpoint.GlobalPoint() + location := v.syncer.checkpoint.FlushedGlobalPoint() err := v.streamerController.Start(tctx, location) if err != nil { v.errChan <- terror.Annotate(err, "fail to start streamer controller") @@ -377,41 +411,35 @@ func (v *DataValidator) doValidate() { return } - if !utils.IsFakeRotateEvent(e.Header) { - // wait until syncer synced that event - err := v.waitSyncerSynced(&currLoc, e) - if err != nil { - v.errChan <- terror.Annotate(err, "failed to wait syncer") - return - } - } - switch ev := e.Event.(type) { case *replication.RotateEvent: currLoc.Position.Name = string(ev.NextLogName) currLoc.Position.Pos = uint32(ev.Position) - case *replication.QueryEvent: - // TODO not processed now + case *replication.GTIDEvent, *replication.MariadbGTIDEvent: currLoc.Position.Pos = e.Header.LogPos - err2 := currLoc.SetGTID(ev.GSet) - if err2 != nil { - v.errChan <- terror.Annotate(err2, "failed to set gtid") + gtidStr, _ := event.GetGTIDStr(e) + if err = currLoc.Update(gtidStr); err != nil { + v.errChan <- terror.Annotate(err, "failed to update gtid set") return } - case *replication.RowsEvent: + default: currLoc.Position.Pos = e.Header.LogPos + } + + // wait until syncer synced current event + err = v.waitSyncerSynced(currLoc) + if err != nil { + v.errChan <- terror.Annotate(err, "failed to wait syncer") + return + } + + switch e.Event.(type) { + case *replication.RowsEvent: select { case v.rowsEventChan <- e: case <-v.ctx.Done(): return } - case *replication.XIDEvent: - currLoc.Position.Pos = e.Header.LogPos - err2 := currLoc.SetGTID(ev.GSet) - if err2 != nil { - v.errChan <- terror.Annotate(err2, "failed to set gtid") - return - } } } } From 6c983439b47b59fcabc6d30f5e8534084f2564c8 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 17 Feb 2022 16:23:57 +0800 Subject: [PATCH 12/36] split worker --- dm/syncer/data_validator.go | 704 +++---------------------------- dm/syncer/data_validator_test.go | 2 +- dm/syncer/validate_worker.go | 494 ++++++++++++++++++++++ 3 files changed, 561 insertions(+), 639 deletions(-) create mode 100644 dm/syncer/validate_worker.go diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index b4bce7d0e44..d5de53e545c 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -15,23 +15,16 @@ package syncer import ( "context" - "database/sql" "fmt" "hash/fnv" - "math" - "sort" - "strconv" "strings" "sync" "time" "github.com/go-mysql-org/go-mysql/replication" "github.com/pingcap/errors" - "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser/model" - tidbmysql "github.com/pingcap/tidb/parser/mysql" - "go.uber.org/atomic" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/config" @@ -50,62 +43,20 @@ import ( const ( checkInterval = 5 * time.Second - retryInterval = 5 * time.Second validationInterval = 10 * time.Second defaultBatchRowCount = 500 - defaultWorkerCnt = 5 + workerChannelSize = 1000 + defaultWorkerCnt = 4 ) type validateTableInfo struct { - Schema string - Name string + Source *filter.Table Info *model.TableInfo PrimaryKey *model.IndexInfo ColumnMap map[string]*model.ColumnInfo Target *filter.Table // target table after route } -type RowDataIterator interface { - // Next seeks the next row data, it used when compared rows. - Next() (map[string]*dbutil.ColumnData, error) - // Close release the resource. - Close() -} - -type RowDataIteratorImpl struct { - Rows *sql.Rows -} - -func (r *RowDataIteratorImpl) Next() (map[string]*dbutil.ColumnData, error) { - for r.Rows.Next() { - rowData, err := dbutil.ScanRow(r.Rows) - return rowData, err - } - return nil, nil -} - -func (r *RowDataIteratorImpl) Close() { - r.Rows.Close() -} - -type RowChangeIteratorImpl struct { - Rows []map[string]*dbutil.ColumnData - Idx int -} - -func (b *RowChangeIteratorImpl) Next() (map[string]*dbutil.ColumnData, error) { - if b.Idx >= len(b.Rows) { - return nil, nil - } - row := b.Rows[b.Idx] - b.Idx++ - return row, nil -} - -func (b *RowChangeIteratorImpl) Close() { - // skip: nothing to do -} - type rowChangeType int const ( @@ -125,6 +76,7 @@ type tableChange struct { // change of a row type rowChange struct { + table *validateTableInfo pk []string data []interface{} theType rowChangeType @@ -154,28 +106,20 @@ type DataValidator struct { L log.Logger fromDB *conn.BaseDB - fromDBConn *dbconn.DBConn toDB *conn.BaseDB - toDBConn *dbconn.DBConn + toDBConns []*dbconn.DBConn timezone *time.Location syncCfg replication.BinlogSyncerConfig streamerController *StreamerController - result pb.ProcessResult - batchRowCnt int - retryInterval time.Duration - - failedRowCnt atomic.Int64 - accumulatedChanges map[string]*tableChange - pendingRowCnt atomic.Int64 - rowsEventChan chan *replication.BinlogEvent // unbuffered is enough - pendingChangeCh chan map[string]*tableChange - changeEventCount []int - validationTimer *time.Timer - tables map[string]*validateTableInfo - workerCnt int - pendingChangeChs map[int]chan map[string]*tableChange // replace pendingChangeCh - failedChangesMap map[int]map[string]*tableChange // replace failedChanges + result pb.ProcessResult + batchRowCnt int + validateInterval time.Duration + workers []*validateWorker + rowsEventChan chan *replication.BinlogEvent // unbuffered is enough + changeEventCount []int + tables map[string]*validateTableInfo + workerCnt int // such as table without primary key unsupportedTable map[string]string @@ -183,33 +127,25 @@ type DataValidator struct { } func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *DataValidator { - c := &DataValidator{ + v := &DataValidator{ cfg: cfg, syncer: syncerObj, stage: pb.Stage_Stopped, errChan: make(chan error, 1), } - c.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) - - c.workerCnt = defaultWorkerCnt - c.pendingChangeChs = make(map[int]chan map[string]*tableChange) - c.failedChangesMap = make(map[int]map[string]*tableChange) - for i := 0; i < c.workerCnt; i++ { - c.pendingChangeChs[i] = make(chan map[string]*tableChange) - c.failedChangesMap[i] = make(map[string]*tableChange) - } - c.batchRowCnt = defaultBatchRowCount - c.validationTimer = time.NewTimer(validationInterval) - c.rowsEventChan = make(chan *replication.BinlogEvent) - c.pendingChangeCh = make(chan map[string]*tableChange) - c.tables = make(map[string]*validateTableInfo) - c.changeEventCount = make([]int, 4) - c.accumulatedChanges = make(map[string]*tableChange) - c.retryInterval = retryInterval - - c.waitSyncerTimer = utils.NewStoppedTimer() - - return c + v.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) + + v.workerCnt = defaultWorkerCnt + v.batchRowCnt = defaultBatchRowCount + v.workers = make([]*validateWorker, v.workerCnt) + v.rowsEventChan = make(chan *replication.BinlogEvent) + v.tables = make(map[string]*validateTableInfo) + v.changeEventCount = make([]int, 4) + v.validateInterval = validationInterval + + v.waitSyncerTimer = utils.NewStoppedTimer() + + return v } func (v *DataValidator) initialize() error { @@ -222,28 +158,21 @@ func (v *DataValidator) initialize() error { if err == nil { return } - if v.fromDB != nil { - v.fromDB.Close() - } - if v.toDB != nil { - v.toDB.Close() - } + dbconn.CloseBaseDB(tctx, v.fromDB) + dbconn.CloseBaseDB(tctx, v.toDB) }() dbCfg := v.cfg.From dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxDMLConnectionTimeout) - var fromDBConns, toDBConns []*dbconn.DBConn - v.fromDB, fromDBConns, err = dbconn.CreateConns(tctx, v.cfg, &dbCfg, 1) + v.fromDB, _, err = dbconn.CreateConns(tctx, v.cfg, &dbCfg, 0) if err != nil { return err } - v.fromDBConn = fromDBConns[0] - v.toDB, toDBConns, err = dbconn.CreateConns(tctx, v.cfg, &dbCfg, 1) + v.toDB, v.toDBConns, err = dbconn.CreateConns(tctx, v.cfg, &dbCfg, v.workerCnt) if err != nil { return err } - v.toDBConn = toDBConns[0] v.timezone, err = str2TimezoneOrFromDB(tctx, v.cfg.Timezone, &v.cfg.To) if err != nil { @@ -399,9 +328,7 @@ func (v *DataValidator) doValidate() { } v.L.Info("start continuous validation") - v.wg.Add(2) - go v.rowsEventProcessRoutine() - go v.validateTaskDispatchRoutine() + v.startValidateWorkers() var currLoc binlog.Location for { @@ -433,11 +360,11 @@ func (v *DataValidator) doValidate() { return } - switch e.Event.(type) { + switch ev := e.Event.(type) { case *replication.RowsEvent: - select { - case v.rowsEventChan <- e: - case <-v.ctx.Done(): + if err = v.processEventRows(e.Header, ev); err != nil { + v.L.Warn("failed to process event: ", zap.Reflect("error", err)) + v.errChan <- terror.Annotate(err, "failed to process event") return } } @@ -478,70 +405,23 @@ func (v *DataValidator) Stage() pb.Stage { return v.stage } -func (v *DataValidator) rowsEventProcessRoutine() { - defer v.wg.Done() - for { - select { - case <-v.ctx.Done(): - v.L.Debug("validator row event process unit done") - return - case e := <-v.rowsEventChan: - if err := v.processEventRows(e.Header, e.Event.(*replication.RowsEvent)); err != nil { - v.L.Warn("failed to process event: ", zap.Reflect("error", err)) - v.errChan <- terror.Annotate(err, "failed to process event") - return - } - case <-v.validationTimer.C: - rowCount := v.getRowCount(v.accumulatedChanges) - if rowCount > 0 { - v.pendingChangeCh <- v.accumulatedChanges - v.accumulatedChanges = make(map[string]*tableChange) - } - v.validationTimer.Reset(validationInterval) - } - } -} - -func (v *DataValidator) validateTableChangeWorkerRoutine(workerID int) { - defer v.wg.Done() - for { - select { - case change := <-v.pendingChangeChs[workerID]: - // 1. validate table change - // 2. update failed rows - // 3. update pending row count - // 4. update failed row cnt - failed := v.validateTableChange(change) - deltaFailed := v.updateFailedChangesByWorker(change, failed, workerID) - v.pendingRowCnt.Sub(int64(v.getRowCount(change))) // atomic value - v.failedRowCnt.Add(int64(deltaFailed)) // atomic value - case <-v.ctx.Done(): - v.L.Debug("validator worker done", zap.Int("workerID", workerID)) - return - case <-time.After(v.retryInterval): - retryChange := v.failedChangesMap[workerID] - failed := v.validateTableChange(retryChange) - deltaFailed := v.updateFailedChangesByWorker(retryChange, failed, workerID) - v.failedRowCnt.Add(int64(deltaFailed)) - } - } -} - -func (v *DataValidator) validateTaskDispatchRoutine() { - defer v.wg.Done() - // start workers here +func (v *DataValidator) startValidateWorkers() { v.wg.Add(v.workerCnt) for i := 0; i < v.workerCnt; i++ { - go v.validateTableChangeWorkerRoutine(i) - } - for { - select { - case change := <-v.pendingChangeCh: - v.dispatchRowChange(change) // dispatch change to worker - case <-v.ctx.Done(): - v.L.Debug("validator task dispatch done") - return + worker := &validateWorker{ + ctx: v.ctx, + interval: v.validateInterval, + validator: v, + L: v.L, + conn: v.toDBConns[i], + rowChangeCh: make(chan *rowChange, workerChannelSize), + pendingChangesMap: make(map[string]*tableChange), } + v.workers[i] = worker + go func() { + v.wg.Done() + worker.run() + }() } } @@ -551,33 +431,14 @@ func hashTablePk(s string) uint32 { return h.Sum32() } -func (v *DataValidator) dispatchRowChange(change map[string]*tableChange) { - dispatchMap := make(map[int]map[string]*tableChange, 0) - for tableName := range change { - // for every table - for _, curRowChange := range change[tableName].rows { - // for every row in the table - // 1. join primary key by '-' - // 2. hash (tableName, primaryKey) to hashVal - // 3. dispatch the row change to dispatchMap[hashVal][tableName] - pk := strings.Join(curRowChange.pk, "-") - hashKey := tableName + "," + pk - hashVal := int(hashTablePk(hashKey)) % v.workerCnt - if _, ok := dispatchMap[hashVal]; !ok { - dispatchMap[hashVal] = make(map[string]*tableChange, 0) - } - if _, ok := dispatchMap[hashVal][tableName]; !ok { - dispatchMap[hashVal][tableName] = &tableChange{ - table: change[tableName].table, - rows: make(map[string]*rowChange, 0), - } - } - dispatchMap[hashVal][tableName].rows[pk] = curRowChange - } - } - for hashVal := range dispatchMap { - v.pendingChangeChs[hashVal] <- dispatchMap[hashVal] - } +func (v *DataValidator) dispatchRowChange(key string, row *rowChange) { + // for every row in the table + // 1. join primary key by '-' + // 2. hash (tableName, primaryKey) to hashVal + // 3. dispatch the row change to dispatchMap[hashVal][tableName] + hashKey := strings.Join([]string{row.table.Source.String(), key}, "-") + hashVal := int(hashTablePk(hashKey)) % v.workerCnt + v.workers[hashVal].rowChangeCh <- row } func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *replication.RowsEvent) error { @@ -617,8 +478,7 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re } table := &validateTableInfo{ - Schema: sourceTable.Schema, - Name: sourceTable.Name, + Source: sourceTable, Info: tableInfo, PrimaryKey: primaryIdx, ColumnMap: columnMap, @@ -627,7 +487,7 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re for _, cols := range ev.SkippedColumns { if len(cols) > 0 { - err := errors.Errorf("unexpected skipped columns for table `%s`.`%s`", table.Schema, table.Name) + err := errors.Errorf("unexpected skipped columns for table %s", sourceTable.String()) return err } } @@ -641,35 +501,11 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re pkIndices[i] = table.ColumnMap[col.Name.O].Offset } - rowCount := v.getRowCount(v.accumulatedChanges) - change := v.accumulatedChanges[fullTableName] - - updateRowChange := func(key string, row *rowChange) { - if val, ok := change.rows[key]; ok { - val.data = row.data - val.theType = row.theType - val.lastMeetTs = row.lastMeetTs - val.retryCnt = row.retryCnt - } else { - change.rows[key] = row - rowCount++ - v.pendingRowCnt.Inc() - } - } - step := 1 if changeType == rowUpdated { step = 2 } for i := 0; i < len(ev.Rows); i += step { - if change == nil { - // no change of this table - change = &tableChange{ - table: table, - rows: make(map[string]*rowChange), - } - v.accumulatedChanges[fullTableName] = change - } row := ev.Rows[i] pkValue := make([]string, len(pk.Columns)) for _, idx := range pkIndices { @@ -687,7 +523,8 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re afterKey := strings.Join(afterPkValue, "-") if afterKey != key { // convert to delete and insert - updateRowChange(key, &rowChange{ + v.dispatchRowChange(key, &rowChange{ + table: table, pk: pkValue, data: row, theType: rowDeleted, @@ -695,33 +532,22 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re }) afterRowChangeType = rowInsert } - updateRowChange(afterKey, &rowChange{ + v.dispatchRowChange(afterKey, &rowChange{ + table: table, pk: afterPkValue, data: afterRow, theType: afterRowChangeType, lastMeetTs: int64(header.Timestamp), }) } else { - updateRowChange(key, &rowChange{ + v.dispatchRowChange(key, &rowChange{ + table: table, pk: pkValue, data: row, theType: changeType, lastMeetTs: int64(header.Timestamp), }) } - - if rowCount >= v.batchRowCnt { - v.pendingChangeCh <- v.accumulatedChanges - v.accumulatedChanges = make(map[string]*tableChange) - - if !v.validationTimer.Stop() { - <-v.validationTimer.C - } - v.validationTimer.Reset(validationInterval) - - rowCount = 0 - change = nil - } } return nil } @@ -746,401 +572,3 @@ func getRowChangeType(t replication.EventType) rowChangeType { return rowDeleted } } - -func (v *DataValidator) validateTableChange(tableChanges map[string]*tableChange) map[string]*tableChange { - failedChanges := make(map[string]*tableChange) - for k, val := range tableChanges { - var insertUpdateChanges, deleteChanges []*rowChange - for _, r := range val.rows { - if r.theType == rowDeleted { - deleteChanges = append(deleteChanges, r) - } else { - insertUpdateChanges = append(insertUpdateChanges, r) - } - } - rows := make(map[string]*rowChange, 0) - if len(insertUpdateChanges) > 0 { - failedRows := v.validateChanges(val.table, insertUpdateChanges, false) - for _, pk := range failedRows { - key := strings.Join(pk, "-") - rows[key] = val.rows[key] - } - } - if len(deleteChanges) > 0 { - failedRows := v.validateChanges(val.table, deleteChanges, true) - for _, pk := range failedRows { - key := strings.Join(pk, "-") - rows[key] = val.rows[key] - } - } - if len(rows) > 0 { - failedChanges[k] = &tableChange{ - table: val.table, - rows: rows, - } - } - } - return failedChanges -} - -func (v *DataValidator) validateChanges(table *validateTableInfo, rows []*rowChange, deleteChange bool) [][]string { - pkValues := make([][]string, 0, len(rows)) - for _, r := range rows { - pkValues = append(pkValues, r.pk) - } - cond := &Cond{Table: table, PkValues: pkValues} - var failedRows [][]string - var err error - if deleteChange { - failedRows, err = v.validateDeletedRows(cond) - } else { - failedRows, err = v.validateInsertAndUpdateRows(rows, cond) - } - if err != nil { - v.L.Warn("fail to validate row changes of table", zap.Error(err)) - return [][]string{} - } - return failedRows -} - -// remove previous failed rows related to current batch of rows -// This function updates the failed rows every time after validating -func (v *DataValidator) updateFailedChangesByWorker(all, failed map[string]*tableChange, workerID int) int { - failedChanges := v.failedChangesMap[workerID] - deltaFailed := 0 - for k, val := range all { - // remove from all - prevFailed := failedChanges[k] - if prevFailed == nil { - continue - } - for _, r := range val.rows { - key := strings.Join(r.pk, "-") - if _, ok := prevFailed.rows[key]; ok { - delete(prevFailed.rows, key) - deltaFailed-- - } - } - } - for k, val := range failed { - // add from failed - prevFailed := failedChanges[k] - if prevFailed == nil { - prevFailed = &tableChange{ - table: val.table, - rows: make(map[string]*rowChange), - } - failedChanges[k] = prevFailed - } - - for _, r := range val.rows { - key := strings.Join(r.pk, "-") - prevFailed.rows[key] = r - deltaFailed++ - } - } - return deltaFailed -} - -func (v *DataValidator) validateDeletedRows(cond *Cond) ([][]string, error) { - downstreamRowsIterator, err := getRowsFrom(cond, v.toDBConn) - if err != nil { - return [][]string{}, err - } - defer downstreamRowsIterator.Close() - - var failedRows [][]string - for { - data, err := downstreamRowsIterator.Next() - if err != nil { - return nil, err - } - if data == nil { - break - } - failedRows = append(failedRows, getPKValues(data, cond)) - } - return failedRows, nil -} - -func (v *DataValidator) validateInsertAndUpdateRows(rows []*rowChange, cond *Cond) ([][]string, error) { - var failedRows [][]string - var upstreamRowsIterator, downstreamRowsIterator RowDataIterator - var err error - upstreamRowsIterator, err = getRowChangeIterator(cond.Table, rows) - if err != nil { - return nil, err - } - defer upstreamRowsIterator.Close() - downstreamRowsIterator, err = getRowsFrom(cond, v.toDBConn) - if err != nil { - return nil, err - } - defer downstreamRowsIterator.Close() - - var lastUpstreamData, lastDownstreamData map[string]*dbutil.ColumnData - - tableInfo := cond.Table.Info - _, orderKeyCols := dbutil.SelectUniqueOrderKey(tableInfo) - for { - if lastUpstreamData == nil { - lastUpstreamData, err = upstreamRowsIterator.Next() - if err != nil { - return nil, err - } - } - - if lastDownstreamData == nil { - lastDownstreamData, err = downstreamRowsIterator.Next() - if err != nil { - return nil, err - } - } - - // may have deleted on upstream and haven't synced to downstream, - // we mark this as success as we'll check the delete-event later - // or downstream removed the pk and added more data by other clients, skip it. - if lastUpstreamData == nil && lastDownstreamData != nil { - v.L.Debug("more data on downstream, may come from other client, skip it") - break - } - - if lastDownstreamData == nil { - // target lack some data, should insert the last source datas - for lastUpstreamData != nil { - failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) - - lastUpstreamData, err = upstreamRowsIterator.Next() - if err != nil { - return nil, err - } - } - break - } - - eq, cmp, err := v.compareData(lastUpstreamData, lastDownstreamData, orderKeyCols, tableInfo.Columns) - if err != nil { - return nil, err - } - if eq { - lastDownstreamData = nil - lastUpstreamData = nil - continue - } - - switch cmp { - case 1: - // may have deleted on upstream and haven't synced to downstream, - // we mark this as success as we'll check the delete-event later - // or downstream removed the pk and added more data by other clients, skip it. - v.L.Debug("more data on downstream, may come from other client, skip it", zap.Reflect("data", lastDownstreamData)) - lastDownstreamData = nil - case -1: - failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) - lastUpstreamData = nil - case 0: - failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) - lastUpstreamData = nil - lastDownstreamData = nil - } - } - return failedRows, nil -} - -func getRowsFrom(cond *Cond, conn *dbconn.DBConn) (RowDataIterator, error) { - tctx := tcontext.NewContext(context.Background(), log.L()) - fullTableName := dbutil.TableName(cond.Table.Schema, cond.Table.Name) - orderKeys, _ := dbutil.SelectUniqueOrderKey(cond.Table.Info) - columnNames := make([]string, 0, len(cond.Table.Info.Columns)) - for _, col := range cond.Table.ColumnMap { - columnNames = append(columnNames, dbutil.ColumnName(col.Name.O)) - } - columns := strings.Join(columnNames, ", ") - rowsQuery := fmt.Sprintf("SELECT /*!40001 SQL_NO_CACHE */ %s FROM %s WHERE %s ORDER BY %s", - columns, fullTableName, cond.GetWhere(), strings.Join(orderKeys, ",")) - rows, err := conn.QuerySQL(tctx, rowsQuery, cond.GetArgs()...) - if err != nil { - return nil, err - } - newRowDataIter := &RowDataIteratorImpl{ - Rows: rows, - } - return newRowDataIter, nil -} - -func getRowChangeIterator(table *validateTableInfo, rows []*rowChange) (RowDataIterator, error) { - sort.Slice(rows, func(i, j int) bool { - left, right := rows[i], rows[j] - for idx := range left.pk { - if left.pk[idx] != right.pk[idx] { - return left.pk[idx] < right.pk[idx] - } - } - return false - }) - it := &RowChangeIteratorImpl{} - for _, r := range rows { - colMap := make(map[string]*dbutil.ColumnData) - for _, c := range table.Info.Columns { - var colData []byte - if r.data[c.Offset] != nil { - colData = []byte(fmt.Sprintf("%v", r.data[c.Offset])) - } - colMap[c.Name.O] = &dbutil.ColumnData{ - Data: colData, - IsNull: r.data[c.Offset] == nil, - } - } - it.Rows = append(it.Rows, colMap) - } - return it, nil -} - -func getPKValues(data map[string]*dbutil.ColumnData, cond *Cond) []string { - var pkValues []string - for _, pkColumn := range cond.Table.PrimaryKey.Columns { - // TODO primary key cannot be null, if we uses unique key should make sure all columns are not null - pkValues = append(pkValues, string(data[pkColumn.Name.O].Data)) - } - return pkValues -} - -func (v *DataValidator) compareData(map1, map2 map[string]*dbutil.ColumnData, orderKeyCols, columns []*model.ColumnInfo) (equal bool, cmp int32, err error) { - var ( - data1, data2 *dbutil.ColumnData - str1, str2 string - key string - ok bool - ) - - equal = true - - defer func() { - if equal || err != nil { - return - } - - if cmp == 0 { - v.L.Warn("find different row", zap.String("column", key), zap.String("row1", rowToString(map1)), zap.String("row2", rowToString(map2))) - } else if cmp > 0 { - v.L.Warn("target had superfluous data", zap.String("row", rowToString(map2))) - } else { - v.L.Warn("target lack data", zap.String("row", rowToString(map1))) - } - }() - - for _, column := range columns { - if data1, ok = map1[column.Name.O]; !ok { - return false, 0, errors.Errorf("upstream doesn't have key %s", key) - } - if data2, ok = map2[column.Name.O]; !ok { - return false, 0, errors.Errorf("downstream doesn't have key %s", key) - } - str1 = string(data1.Data) - str2 = string(data2.Data) - if column.FieldType.Tp == tidbmysql.TypeFloat || column.FieldType.Tp == tidbmysql.TypeDouble { - if data1.IsNull == data2.IsNull && data1.IsNull { - continue - } - - num1, err1 := strconv.ParseFloat(str1, 64) - num2, err2 := strconv.ParseFloat(str2, 64) - if err1 != nil || err2 != nil { - err = errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", str1, str2, err1, err2) - return - } - if math.Abs(num1-num2) <= 1e-6 { - continue - } - } else { - if (str1 == str2) && (data1.IsNull == data2.IsNull) { - continue - } - } - - equal = false - break - - } - if equal { - return - } - - // Not Equal. Compare orderkeycolumns. - for _, col := range orderKeyCols { - if data1, ok = map1[col.Name.O]; !ok { - err = errors.Errorf("upstream doesn't have key %s", col.Name.O) - return - } - if data2, ok = map2[col.Name.O]; !ok { - err = errors.Errorf("downstream doesn't have key %s", col.Name.O) - return - } - - if NeedQuotes(col.FieldType.Tp) { - strData1 := string(data1.Data) - strData2 := string(data2.Data) - - if len(strData1) == len(strData2) && strData1 == strData2 { - continue - } - - if strData1 < strData2 { - cmp = -1 - } else { - cmp = 1 - } - break - } else if data1.IsNull || data2.IsNull { - if data1.IsNull && data2.IsNull { - continue - } - - if data1.IsNull { - cmp = -1 - } else { - cmp = 1 - } - break - } else { - num1, err1 := strconv.ParseFloat(string(data1.Data), 64) - num2, err2 := strconv.ParseFloat(string(data2.Data), 64) - if err1 != nil || err2 != nil { - err = errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", string(data1.Data), string(data2.Data), err1, err2) - return - } - - if num1 == num2 { - continue - } - - if num1 < num2 { - cmp = -1 - } else { - cmp = 1 - } - break - } - } - - return -} - -func NeedQuotes(tp byte) bool { - return !(dbutil.IsNumberType(tp) || dbutil.IsFloatType(tp)) -} - -func rowToString(row map[string]*dbutil.ColumnData) string { - var s strings.Builder - s.WriteString("{ ") - for key, val := range row { - if val.IsNull { - s.WriteString(fmt.Sprintf("%s: IsNull, ", key)) - } else { - s.WriteString(fmt.Sprintf("%s: %s, ", key, val.Data)) - } - } - s.WriteString(" }") - - return s.String() -} diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index d122e6cff33..845b1c84e9b 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -642,7 +642,7 @@ func (d *testDataValidatorSuite) TestDoValidate(c *C) { validator.Start(pb.Stage_Paused) // init but will return soon validator.result = pb.ProcessResult{} // clear error validator.workerCnt = 1 - validator.retryInterval = 100 * time.Second // never retry + validator.validateInterval = 100 * time.Second // never retry events1 := testCase.binlogEvs mockStreamerProducer := &MockStreamProducer{d.generateEvents(events1, c)} mockStreamer, err := mockStreamerProducer.generateStreamer(binlog.NewLocation("")) diff --git a/dm/syncer/validate_worker.go b/dm/syncer/validate_worker.go new file mode 100644 index 00000000000..0d0bd773f05 --- /dev/null +++ b/dm/syncer/validate_worker.go @@ -0,0 +1,494 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package syncer + +import ( + "context" + "database/sql" + "fmt" + "math" + "sort" + "strconv" + "strings" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb/parser/model" + tidbmysql "github.com/pingcap/tidb/parser/mysql" + "go.uber.org/zap" + + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/syncer/dbconn" +) + +type RowDataIterator interface { + // Next seeks the next row data, it used when compared rows. + Next() (map[string]*dbutil.ColumnData, error) + // Close release the resource. + Close() +} + +type RowDataIteratorImpl struct { + Rows *sql.Rows +} + +func (r *RowDataIteratorImpl) Next() (map[string]*dbutil.ColumnData, error) { + for r.Rows.Next() { + rowData, err := dbutil.ScanRow(r.Rows) + return rowData, err + } + return nil, nil +} + +func (r *RowDataIteratorImpl) Close() { + r.Rows.Close() +} + +type RowChangeIteratorImpl struct { + Rows []map[string]*dbutil.ColumnData + Idx int +} + +func (b *RowChangeIteratorImpl) Next() (map[string]*dbutil.ColumnData, error) { + if b.Idx >= len(b.Rows) { + return nil, nil + } + row := b.Rows[b.Idx] + b.Idx++ + return row, nil +} + +func (b *RowChangeIteratorImpl) Close() { + // skip: nothing to do +} + +type validateWorker struct { + ctx context.Context + interval time.Duration + validator *DataValidator + L log.Logger + conn *dbconn.DBConn + rowChangeCh chan *rowChange + pendingChangesMap map[string]*tableChange + rowCount int64 +} + +func (vw *validateWorker) run() { + for { + select { + case change := <-vw.rowChangeCh: + // todo: limit number of pending rows + vw.updateRowChange(change) + case <-vw.ctx.Done(): + return + case <-time.After(vw.interval): + err := vw.validateTableChange() + if err != nil { + vw.validator.errChan <- terror.Annotate(err, "failed to validate table change") + return + } + } + } +} + +func (vw *validateWorker) updateRowChange(row *rowChange) { + // cluster using target table + fullTableName := row.table.Target.String() + change := vw.pendingChangesMap[fullTableName] + if change == nil { + // no change of this table + change = &tableChange{ + table: row.table, + rows: make(map[string]*rowChange), + } + vw.pendingChangesMap[fullTableName] = change + } + key := strings.Join(row.pk, "-") + if val, ok := change.rows[key]; ok { + val.data = row.data + val.theType = row.theType + val.lastMeetTs = row.lastMeetTs + val.retryCnt = row.retryCnt + } else { + change.rows[key] = row + vw.rowCount++ + } +} + +func (vw *validateWorker) validateTableChange() error { + failedChanges := make(map[string]*tableChange) + for k, val := range vw.pendingChangesMap { + var insertUpdateChanges, deleteChanges []*rowChange + for _, r := range val.rows { + if r.theType == rowDeleted { + deleteChanges = append(deleteChanges, r) + } else { + insertUpdateChanges = append(insertUpdateChanges, r) + } + } + rows := make(map[string]*rowChange, 0) + if len(insertUpdateChanges) > 0 { + failedRows, err := vw.validateRowChanges(val.table, insertUpdateChanges, false) + if err != nil { + return err + } + for _, pk := range failedRows { + rows[pk] = val.rows[pk] + } + } + if len(deleteChanges) > 0 { + failedRows, err := vw.validateRowChanges(val.table, deleteChanges, true) + if err != nil { + return err + } + for _, pk := range failedRows { + rows[pk] = val.rows[pk] + } + } + if len(rows) > 0 { + failedChanges[k] = &tableChange{ + table: val.table, + rows: rows, + } + } + } + vw.pendingChangesMap = failedChanges + return nil +} + +func (vw *validateWorker) validateRowChanges(table *validateTableInfo, rows []*rowChange, deleteChange bool) ([]string, error) { + pkValues := make([][]string, 0, len(rows)) + for _, r := range rows { + pkValues = append(pkValues, r.pk) + } + cond := &Cond{Table: table, PkValues: pkValues} + var failedRows []string + var err error + if deleteChange { + failedRows, err = vw.validateDeletedRows(cond) + } else { + failedRows, err = vw.validateInsertAndUpdateRows(rows, cond) + } + if err != nil { + vw.L.Warn("fail to validate row changes of table", zap.Error(err)) + return nil, err + } + return failedRows, nil +} + +func (vw *validateWorker) validateDeletedRows(cond *Cond) ([]string, error) { + downstreamRowsIterator, err := vw.getRowsFrom(cond) + if err != nil { + return []string{}, err + } + defer downstreamRowsIterator.Close() + + var failedRows []string + for { + data, err := downstreamRowsIterator.Next() + if err != nil { + return nil, err + } + if data == nil { + break + } + failedRows = append(failedRows, getPKValues(data, cond)) + } + return failedRows, nil +} + +func (vw *validateWorker) validateInsertAndUpdateRows(rows []*rowChange, cond *Cond) ([]string, error) { + var failedRows []string + var upstreamRowsIterator, downstreamRowsIterator RowDataIterator + var err error + upstreamRowsIterator, err = getRowChangeIterator(cond.Table, rows) + if err != nil { + return nil, err + } + defer upstreamRowsIterator.Close() + downstreamRowsIterator, err = vw.getRowsFrom(cond) + if err != nil { + return nil, err + } + defer downstreamRowsIterator.Close() + + var lastUpstreamData, lastDownstreamData map[string]*dbutil.ColumnData + + tableInfo := cond.Table.Info + _, orderKeyCols := dbutil.SelectUniqueOrderKey(tableInfo) + for { + if lastUpstreamData == nil { + lastUpstreamData, err = upstreamRowsIterator.Next() + if err != nil { + return nil, err + } + } + + if lastDownstreamData == nil { + lastDownstreamData, err = downstreamRowsIterator.Next() + if err != nil { + return nil, err + } + } + + // may have deleted on upstream and haven't synced to downstream, + // we mark this as success as we'll check the delete-event later + // or downstream removed the pk and added more data by other clients, skip it. + if lastUpstreamData == nil && lastDownstreamData != nil { + vw.L.Debug("more data on downstream, may come from other client, skip it") + break + } + + if lastDownstreamData == nil { + // target lack some data, should insert the last source datas + for lastUpstreamData != nil { + failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) + + lastUpstreamData, err = upstreamRowsIterator.Next() + if err != nil { + return nil, err + } + } + break + } + + eq, cmp, err := vw.compareData(lastUpstreamData, lastDownstreamData, orderKeyCols, tableInfo.Columns) + if err != nil { + return nil, err + } + if eq { + lastDownstreamData = nil + lastUpstreamData = nil + continue + } + + switch cmp { + case 1: + // may have deleted on upstream and haven't synced to downstream, + // we mark this as success as we'll check the delete-event later + // or downstream removed the pk and added more data by other clients, skip it. + vw.L.Debug("more data on downstream, may come from other client, skip it", zap.Reflect("data", lastDownstreamData)) + lastDownstreamData = nil + case -1: + failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) + lastUpstreamData = nil + case 0: + failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) + lastUpstreamData = nil + lastDownstreamData = nil + } + } + return failedRows, nil +} + +func (vw *validateWorker) compareData(map1, map2 map[string]*dbutil.ColumnData, orderKeyCols, columns []*model.ColumnInfo) (equal bool, cmp int32, err error) { + var ( + data1, data2 *dbutil.ColumnData + str1, str2 string + key string + ok bool + ) + + equal = true + + defer func() { + if equal || err != nil { + return + } + + if cmp == 0 { + vw.L.Warn("find different row", zap.String("column", key), zap.String("row1", rowToString(map1)), zap.String("row2", rowToString(map2))) + } else if cmp > 0 { + vw.L.Warn("target had superfluous data", zap.String("row", rowToString(map2))) + } else { + vw.L.Warn("target lack data", zap.String("row", rowToString(map1))) + } + }() + + for _, column := range columns { + if data1, ok = map1[column.Name.O]; !ok { + return false, 0, errors.Errorf("upstream doesn't have key %s", key) + } + if data2, ok = map2[column.Name.O]; !ok { + return false, 0, errors.Errorf("downstream doesn't have key %s", key) + } + str1 = string(data1.Data) + str2 = string(data2.Data) + if column.FieldType.Tp == tidbmysql.TypeFloat || column.FieldType.Tp == tidbmysql.TypeDouble { + if data1.IsNull == data2.IsNull && data1.IsNull { + continue + } + + num1, err1 := strconv.ParseFloat(str1, 64) + num2, err2 := strconv.ParseFloat(str2, 64) + if err1 != nil || err2 != nil { + err = errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", str1, str2, err1, err2) + return + } + if math.Abs(num1-num2) <= 1e-6 { + continue + } + } else { + if (str1 == str2) && (data1.IsNull == data2.IsNull) { + continue + } + } + + equal = false + break + + } + if equal { + return + } + + // Not Equal. Compare orderkeycolumns. + for _, col := range orderKeyCols { + if data1, ok = map1[col.Name.O]; !ok { + err = errors.Errorf("upstream doesn't have key %s", col.Name.O) + return + } + if data2, ok = map2[col.Name.O]; !ok { + err = errors.Errorf("downstream doesn't have key %s", col.Name.O) + return + } + + if NeedQuotes(col.FieldType.Tp) { + strData1 := string(data1.Data) + strData2 := string(data2.Data) + + if len(strData1) == len(strData2) && strData1 == strData2 { + continue + } + + if strData1 < strData2 { + cmp = -1 + } else { + cmp = 1 + } + break + } else if data1.IsNull || data2.IsNull { + if data1.IsNull && data2.IsNull { + continue + } + + if data1.IsNull { + cmp = -1 + } else { + cmp = 1 + } + break + } else { + num1, err1 := strconv.ParseFloat(string(data1.Data), 64) + num2, err2 := strconv.ParseFloat(string(data2.Data), 64) + if err1 != nil || err2 != nil { + err = errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", string(data1.Data), string(data2.Data), err1, err2) + return + } + + if num1 == num2 { + continue + } + + if num1 < num2 { + cmp = -1 + } else { + cmp = 1 + } + break + } + } + + return +} + +func (vw *validateWorker) getRowsFrom(cond *Cond) (RowDataIterator, error) { + tctx := tcontext.NewContext(context.Background(), log.L()) + fullTableName := cond.Table.Target.String() + orderKeys, _ := dbutil.SelectUniqueOrderKey(cond.Table.Info) + columnNames := make([]string, 0, len(cond.Table.Info.Columns)) + for _, col := range cond.Table.ColumnMap { + columnNames = append(columnNames, dbutil.ColumnName(col.Name.O)) + } + columns := strings.Join(columnNames, ", ") + rowsQuery := fmt.Sprintf("SELECT /*!40001 SQL_NO_CACHE */ %s FROM %s WHERE %s ORDER BY %s", + columns, fullTableName, cond.GetWhere(), strings.Join(orderKeys, ",")) + rows, err := vw.conn.QuerySQL(tctx, rowsQuery, cond.GetArgs()...) + if err != nil { + return nil, err + } + newRowDataIter := &RowDataIteratorImpl{ + Rows: rows, + } + return newRowDataIter, nil +} + +func getRowChangeIterator(table *validateTableInfo, rows []*rowChange) (RowDataIterator, error) { + sort.Slice(rows, func(i, j int) bool { + left, right := rows[i], rows[j] + for idx := range left.pk { + if left.pk[idx] != right.pk[idx] { + return left.pk[idx] < right.pk[idx] + } + } + return false + }) + it := &RowChangeIteratorImpl{} + for _, r := range rows { + colMap := make(map[string]*dbutil.ColumnData) + for _, c := range table.Info.Columns { + var colData []byte + if r.data[c.Offset] != nil { + colData = []byte(fmt.Sprintf("%v", r.data[c.Offset])) + } + colMap[c.Name.O] = &dbutil.ColumnData{ + Data: colData, + IsNull: r.data[c.Offset] == nil, + } + } + it.Rows = append(it.Rows, colMap) + } + return it, nil +} + +func rowToString(row map[string]*dbutil.ColumnData) string { + var s strings.Builder + s.WriteString("{ ") + for key, val := range row { + if val.IsNull { + s.WriteString(fmt.Sprintf("%s: IsNull, ", key)) + } else { + s.WriteString(fmt.Sprintf("%s: %s, ", key, val.Data)) + } + } + s.WriteString(" }") + + return s.String() +} + +func getPKValues(data map[string]*dbutil.ColumnData, cond *Cond) string { + var pkValues []string + for _, pkColumn := range cond.Table.PrimaryKey.Columns { + // TODO primary key cannot be null, if we uses unique key should make sure all columns are not null + pkValues = append(pkValues, string(data[pkColumn.Name.O].Data)) + } + return strings.Join(pkValues, "-") +} + +func NeedQuotes(tp byte) bool { + return !(dbutil.IsNumberType(tp) || dbutil.IsFloatType(tp)) +} From 86cbced723528af56a04b1e40b7c9e60ade90ee0 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Fri, 18 Feb 2022 14:04:19 +0800 Subject: [PATCH 13/36] refactor --- dm/syncer/data_validator.go | 38 ++-- dm/syncer/data_validator_test.go | 26 +-- dm/syncer/validate_worker.go | 376 ++++++++----------------------- dm/syncer/validator_cond.go | 5 +- 4 files changed, 140 insertions(+), 305 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index d5de53e545c..53e3ace07a4 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -53,8 +53,8 @@ type validateTableInfo struct { Source *filter.Table Info *model.TableInfo PrimaryKey *model.IndexInfo - ColumnMap map[string]*model.ColumnInfo Target *filter.Table // target table after route + pkIndices []int } type rowChangeType int @@ -77,11 +77,12 @@ type tableChange struct { // change of a row type rowChange struct { table *validateTableInfo - pk []string + key string + pkValues []string data []interface{} theType rowChangeType lastMeetTs int64 // the last meet timestamp(in seconds) - retryCnt int // retry count + failedCnt int // failed count } // DataValidator @@ -103,6 +104,7 @@ type DataValidator struct { errChan chan error ctx context.Context cancel context.CancelFunc + tctx *tcontext.Context L log.Logger fromDB *conn.BaseDB @@ -198,6 +200,7 @@ func (v *DataValidator) Start(expect pb.Stage) { } v.ctx, v.cancel = context.WithCancel(context.Background()) + v.tctx = tcontext.NewContext(v.ctx, v.L) if err := v.initialize(); err != nil { v.fillResult(err, false) @@ -318,10 +321,9 @@ func (v *DataValidator) doValidate() { return } - tctx := tcontext.NewContext(v.ctx, v.L) // todo: syncer may change replication location(start from timestamp, sharding resync), how validator react? location := v.syncer.checkpoint.FlushedGlobalPoint() - err := v.streamerController.Start(tctx, location) + err := v.streamerController.Start(v.tctx, location) if err != nil { v.errChan <- terror.Annotate(err, "fail to start streamer controller") return @@ -332,7 +334,7 @@ func (v *DataValidator) doValidate() { var currLoc binlog.Location for { - e, err := v.streamerController.GetEvent(tctx) + e, err := v.streamerController.GetEvent(v.tctx) if err != nil { v.errChan <- terror.Annotate(err, "fail to get binlog from stream controller") return @@ -461,10 +463,11 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re return terror.Annotate(err, "failed to get table info") } - columnMap := make(map[string]*model.ColumnInfo) - for _, col := range tableInfo.Columns { - columnMap[col.Name.O] = col + if len(tableInfo.Columns) < int(ev.ColumnCount) { + v.unsupportedTable[fullTableName] = "binlog has more columns than current table" + return nil } + var primaryIdx *model.IndexInfo for _, idx := range tableInfo.Indices { if idx.Primary { @@ -481,7 +484,6 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re Source: sourceTable, Info: tableInfo, PrimaryKey: primaryIdx, - ColumnMap: columnMap, Target: targetTable, } @@ -495,11 +497,16 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re changeType := getRowChangeType(header.EventType) v.changeEventCount[changeType]++ + columnMap := make(map[string]*model.ColumnInfo) + for _, col := range tableInfo.Columns { + columnMap[col.Name.O] = col + } pk := table.PrimaryKey pkIndices := make([]int, len(pk.Columns)) for i, col := range pk.Columns { - pkIndices[i] = table.ColumnMap[col.Name.O].Offset + pkIndices[i] = columnMap[col.Name.O].Offset } + table.pkIndices = pkIndices step := 1 if changeType == rowUpdated { @@ -525,7 +532,8 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re // convert to delete and insert v.dispatchRowChange(key, &rowChange{ table: table, - pk: pkValue, + key: key, + pkValues: pkValue, data: row, theType: rowDeleted, lastMeetTs: int64(header.Timestamp), @@ -534,7 +542,8 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re } v.dispatchRowChange(afterKey, &rowChange{ table: table, - pk: afterPkValue, + key: afterKey, + pkValues: afterPkValue, data: afterRow, theType: afterRowChangeType, lastMeetTs: int64(header.Timestamp), @@ -542,7 +551,8 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re } else { v.dispatchRowChange(key, &rowChange{ table: table, - pk: pkValue, + key: key, + pkValues: pkValue, data: row, theType: changeType, lastMeetTs: int64(header.Timestamp), diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index 845b1c84e9b..149dcdfbf31 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -171,24 +171,24 @@ func (r *testDataValidatorSuite) TestRowChangeIteratorImpl(c *C) { c.Assert(err, IsNil) rows := []*rowChange{ { - pk: []string{"pk", "1"}, - data: []interface{}{"pk", "1", "some data"}, - theType: rowInsert, + pkValues: []string{"pk", "1"}, + data: []interface{}{"pk", "1", "some data"}, + theType: rowInsert, }, { - pk: []string{"pk", "2"}, - data: []interface{}{"pk", "2", "some data"}, - theType: rowDeleted, + pkValues: []string{"pk", "2"}, + data: []interface{}{"pk", "2", "some data"}, + theType: rowDeleted, }, { - pk: []string{"pkg", "2"}, - data: []interface{}{"pkg", "2", "some data"}, - theType: rowDeleted, + pkValues: []string{"pkg", "2"}, + data: []interface{}{"pkg", "2", "some data"}, + theType: rowDeleted, }, { - pk: []string{"pke", "2"}, - data: []interface{}{"pke", "2", "some data"}, - theType: rowUpdated, + pkValues: []string{"pke", "2"}, + data: []interface{}{"pke", "2", "some data"}, + theType: rowUpdated, }, } testCases := []map[string]string{ @@ -221,7 +221,7 @@ func (r *testDataValidatorSuite) TestRowChangeIteratorImpl(c *C) { );` tableDiff := getTableDiff(c, db, "test_it", "test1", creatTbl) var iter RowDataIterator - iter, err = getRowChangeIterator(tableDiff, rows) + iter, err = getSourceRowsForCompare(rows) c.Assert(err, IsNil) for i := 0; i < len(rows); i++ { res, err := iter.Next() diff --git a/dm/syncer/validate_worker.go b/dm/syncer/validate_worker.go index 0d0bd773f05..6ac5aeb2748 100644 --- a/dm/syncer/validate_worker.go +++ b/dm/syncer/validate_worker.go @@ -18,7 +18,6 @@ import ( "database/sql" "fmt" "math" - "sort" "strconv" "strings" "time" @@ -35,47 +34,6 @@ import ( "github.com/pingcap/tiflow/dm/syncer/dbconn" ) -type RowDataIterator interface { - // Next seeks the next row data, it used when compared rows. - Next() (map[string]*dbutil.ColumnData, error) - // Close release the resource. - Close() -} - -type RowDataIteratorImpl struct { - Rows *sql.Rows -} - -func (r *RowDataIteratorImpl) Next() (map[string]*dbutil.ColumnData, error) { - for r.Rows.Next() { - rowData, err := dbutil.ScanRow(r.Rows) - return rowData, err - } - return nil, nil -} - -func (r *RowDataIteratorImpl) Close() { - r.Rows.Close() -} - -type RowChangeIteratorImpl struct { - Rows []map[string]*dbutil.ColumnData - Idx int -} - -func (b *RowChangeIteratorImpl) Next() (map[string]*dbutil.ColumnData, error) { - if b.Idx >= len(b.Rows) { - return nil, nil - } - row := b.Rows[b.Idx] - b.Idx++ - return row, nil -} - -func (b *RowChangeIteratorImpl) Close() { - // skip: nothing to do -} - type validateWorker struct { ctx context.Context interval time.Duration @@ -117,14 +75,13 @@ func (vw *validateWorker) updateRowChange(row *rowChange) { } vw.pendingChangesMap[fullTableName] = change } - key := strings.Join(row.pk, "-") - if val, ok := change.rows[key]; ok { + if val, ok := change.rows[row.key]; ok { val.data = row.data val.theType = row.theType val.lastMeetTs = row.lastMeetTs - val.retryCnt = row.retryCnt + val.failedCnt = 0 // clear failed count } else { - change.rows[key] = row + change.rows[row.key] = row vw.rowCount++ } } @@ -142,21 +99,25 @@ func (vw *validateWorker) validateTableChange() error { } rows := make(map[string]*rowChange, 0) if len(insertUpdateChanges) > 0 { + // todo: limit number of validated rows failedRows, err := vw.validateRowChanges(val.table, insertUpdateChanges, false) if err != nil { return err } for _, pk := range failedRows { rows[pk] = val.rows[pk] + rows[pk].failedCnt++ } } if len(deleteChanges) > 0 { + // todo: limit number of validated rows failedRows, err := vw.validateRowChanges(val.table, deleteChanges, true) if err != nil { return err } for _, pk := range failedRows { rows[pk] = val.rows[pk] + rows[pk].failedCnt++ } } if len(rows) > 0 { @@ -173,9 +134,14 @@ func (vw *validateWorker) validateTableChange() error { func (vw *validateWorker) validateRowChanges(table *validateTableInfo, rows []*rowChange, deleteChange bool) ([]string, error) { pkValues := make([][]string, 0, len(rows)) for _, r := range rows { - pkValues = append(pkValues, r.pk) + pkValues = append(pkValues, r.pkValues) + } + colCnt := len(rows[0].data) + cond := &Cond{ + Table: table, + ColumnCnt: colCnt, + PkValues: pkValues, } - cond := &Cond{Table: table, PkValues: pkValues} var failedRows []string var err error if deleteChange { @@ -191,143 +157,55 @@ func (vw *validateWorker) validateRowChanges(table *validateTableInfo, rows []*r } func (vw *validateWorker) validateDeletedRows(cond *Cond) ([]string, error) { - downstreamRowsIterator, err := vw.getRowsFrom(cond) + targetRows, err := vw.getTargetRows(cond) if err != nil { return []string{}, err } - defer downstreamRowsIterator.Close() - var failedRows []string - for { - data, err := downstreamRowsIterator.Next() - if err != nil { - return nil, err - } - if data == nil { - break - } - failedRows = append(failedRows, getPKValues(data, cond)) + failedRows := make([]string, 0, len(targetRows)) + for key := range targetRows { + failedRows = append(failedRows, key) } return failedRows, nil } func (vw *validateWorker) validateInsertAndUpdateRows(rows []*rowChange, cond *Cond) ([]string, error) { var failedRows []string - var upstreamRowsIterator, downstreamRowsIterator RowDataIterator - var err error - upstreamRowsIterator, err = getRowChangeIterator(cond.Table, rows) + sourceRows := getSourceRowsForCompare(rows) + targetRows, err := vw.getTargetRows(cond) if err != nil { return nil, err } - defer upstreamRowsIterator.Close() - downstreamRowsIterator, err = vw.getRowsFrom(cond) - if err != nil { - return nil, err - } - defer downstreamRowsIterator.Close() - var lastUpstreamData, lastDownstreamData map[string]*dbutil.ColumnData + if len(targetRows) > len(sourceRows) { + // if this happens, downstream should have removed the primary key + vw.L.Debug("more data on downstream, may come from other client") + } tableInfo := cond.Table.Info - _, orderKeyCols := dbutil.SelectUniqueOrderKey(tableInfo) - for { - if lastUpstreamData == nil { - lastUpstreamData, err = upstreamRowsIterator.Next() - if err != nil { - return nil, err - } - } - - if lastDownstreamData == nil { - lastDownstreamData, err = downstreamRowsIterator.Next() - if err != nil { - return nil, err - } - } - - // may have deleted on upstream and haven't synced to downstream, - // we mark this as success as we'll check the delete-event later - // or downstream removed the pk and added more data by other clients, skip it. - if lastUpstreamData == nil && lastDownstreamData != nil { - vw.L.Debug("more data on downstream, may come from other client, skip it") - break - } - - if lastDownstreamData == nil { - // target lack some data, should insert the last source datas - for lastUpstreamData != nil { - failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) - - lastUpstreamData, err = upstreamRowsIterator.Next() - if err != nil { - return nil, err - } - } - break + for key, sourceRow := range sourceRows { + targetRow, ok := targetRows[key] + if !ok { + failedRows = append(failedRows, key) + continue } - eq, cmp, err := vw.compareData(lastUpstreamData, lastDownstreamData, orderKeyCols, tableInfo.Columns) + eq, err := vw.compareData(sourceRow, targetRow, tableInfo.Columns[:cond.ColumnCnt]) if err != nil { return nil, err } - if eq { - lastDownstreamData = nil - lastUpstreamData = nil - continue - } - - switch cmp { - case 1: - // may have deleted on upstream and haven't synced to downstream, - // we mark this as success as we'll check the delete-event later - // or downstream removed the pk and added more data by other clients, skip it. - vw.L.Debug("more data on downstream, may come from other client, skip it", zap.Reflect("data", lastDownstreamData)) - lastDownstreamData = nil - case -1: - failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) - lastUpstreamData = nil - case 0: - failedRows = append(failedRows, getPKValues(lastUpstreamData, cond)) - lastUpstreamData = nil - lastDownstreamData = nil + if !eq { + failedRows = append(failedRows, key) } } return failedRows, nil } -func (vw *validateWorker) compareData(map1, map2 map[string]*dbutil.ColumnData, orderKeyCols, columns []*model.ColumnInfo) (equal bool, cmp int32, err error) { - var ( - data1, data2 *dbutil.ColumnData - str1, str2 string - key string - ok bool - ) - - equal = true - - defer func() { - if equal || err != nil { - return - } - - if cmp == 0 { - vw.L.Warn("find different row", zap.String("column", key), zap.String("row1", rowToString(map1)), zap.String("row2", rowToString(map2))) - } else if cmp > 0 { - vw.L.Warn("target had superfluous data", zap.String("row", rowToString(map2))) - } else { - vw.L.Warn("target lack data", zap.String("row", rowToString(map1))) - } - }() - - for _, column := range columns { - if data1, ok = map1[column.Name.O]; !ok { - return false, 0, errors.Errorf("upstream doesn't have key %s", key) - } - if data2, ok = map2[column.Name.O]; !ok { - return false, 0, errors.Errorf("downstream doesn't have key %s", key) - } - str1 = string(data1.Data) - str2 = string(data2.Data) +func (vw *validateWorker) compareData(sourceData, targetData []*dbutil.ColumnData, columns []*model.ColumnInfo) (bool, error) { + equal := true + for i, column := range columns { + data1, data2 := sourceData[i], targetData[i] + str1, str2 := string(data1.Data), string(data2.Data) if column.FieldType.Tp == tidbmysql.TypeFloat || column.FieldType.Tp == tidbmysql.TypeDouble { if data1.IsNull == data2.IsNull && data1.IsNull { continue @@ -336,8 +214,7 @@ func (vw *validateWorker) compareData(map1, map2 map[string]*dbutil.ColumnData, num1, err1 := strconv.ParseFloat(str1, 64) num2, err2 := strconv.ParseFloat(str2, 64) if err1 != nil || err2 != nil { - err = errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", str1, str2, err1, err2) - return + return false, errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", str1, str2, err1, err2) } if math.Abs(num1-num2) <= 1e-6 { continue @@ -350,145 +227,92 @@ func (vw *validateWorker) compareData(map1, map2 map[string]*dbutil.ColumnData, equal = false break - } - if equal { - return - } - - // Not Equal. Compare orderkeycolumns. - for _, col := range orderKeyCols { - if data1, ok = map1[col.Name.O]; !ok { - err = errors.Errorf("upstream doesn't have key %s", col.Name.O) - return - } - if data2, ok = map2[col.Name.O]; !ok { - err = errors.Errorf("downstream doesn't have key %s", col.Name.O) - return - } - - if NeedQuotes(col.FieldType.Tp) { - strData1 := string(data1.Data) - strData2 := string(data2.Data) - - if len(strData1) == len(strData2) && strData1 == strData2 { - continue - } - - if strData1 < strData2 { - cmp = -1 - } else { - cmp = 1 - } - break - } else if data1.IsNull || data2.IsNull { - if data1.IsNull && data2.IsNull { - continue - } - if data1.IsNull { - cmp = -1 - } else { - cmp = 1 - } - break - } else { - num1, err1 := strconv.ParseFloat(string(data1.Data), 64) - num2, err2 := strconv.ParseFloat(string(data2.Data), 64) - if err1 != nil || err2 != nil { - err = errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", string(data1.Data), string(data2.Data), err1, err2) - return - } - - if num1 == num2 { - continue - } - - if num1 < num2 { - cmp = -1 - } else { - cmp = 1 - } - break - } - } - - return + return equal, nil } -func (vw *validateWorker) getRowsFrom(cond *Cond) (RowDataIterator, error) { +func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.ColumnData, error) { tctx := tcontext.NewContext(context.Background(), log.L()) fullTableName := cond.Table.Target.String() - orderKeys, _ := dbutil.SelectUniqueOrderKey(cond.Table.Info) - columnNames := make([]string, 0, len(cond.Table.Info.Columns)) - for _, col := range cond.Table.ColumnMap { + pkColumnNames := make([]string, 0, len(cond.Table.pkIndices)) + for i := range cond.Table.pkIndices { + pkColumnNames = append(pkColumnNames, cond.Table.Info.Columns[i].Name.O) + } + columnNames := make([]string, 0, cond.ColumnCnt) + for i := 0; i < cond.ColumnCnt; i++ { + col := cond.Table.Info.Columns[i] columnNames = append(columnNames, dbutil.ColumnName(col.Name.O)) } columns := strings.Join(columnNames, ", ") rowsQuery := fmt.Sprintf("SELECT /*!40001 SQL_NO_CACHE */ %s FROM %s WHERE %s ORDER BY %s", - columns, fullTableName, cond.GetWhere(), strings.Join(orderKeys, ",")) + columns, fullTableName, cond.GetWhere(), strings.Join(pkColumnNames, ",")) rows, err := vw.conn.QuerySQL(tctx, rowsQuery, cond.GetArgs()...) if err != nil { return nil, err } - newRowDataIter := &RowDataIteratorImpl{ - Rows: rows, - } - return newRowDataIter, nil -} + defer rows.Close() -func getRowChangeIterator(table *validateTableInfo, rows []*rowChange) (RowDataIterator, error) { - sort.Slice(rows, func(i, j int) bool { - left, right := rows[i], rows[j] - for idx := range left.pk { - if left.pk[idx] != right.pk[idx] { - return left.pk[idx] < right.pk[idx] - } + result := make(map[string][]*dbutil.ColumnData, 0) + for rows.Next() { + rowData, err := ScanRow(rows) + if err != nil { + return nil, err } - return false - }) - it := &RowChangeIteratorImpl{} - for _, r := range rows { - colMap := make(map[string]*dbutil.ColumnData) - for _, c := range table.Info.Columns { - var colData []byte - if r.data[c.Offset] != nil { - colData = []byte(fmt.Sprintf("%v", r.data[c.Offset])) - } - colMap[c.Name.O] = &dbutil.ColumnData{ - Data: colData, - IsNull: r.data[c.Offset] == nil, - } + pkVals := make([]string, 0, len(cond.Table.pkIndices)) + for _, idx := range cond.Table.pkIndices { + colVal := fmt.Sprintf("%v", rowData[idx].Data) + pkVals = append(pkVals, colVal) } - it.Rows = append(it.Rows, colMap) + pk := strings.Join(pkVals, "-") + result[pk] = rowData } - return it, nil + return result, nil } +func ScanRow(rows *sql.Rows) ([]*dbutil.ColumnData, error) { + cols, err := rows.Columns() + if err != nil { + return nil, errors.Trace(err) + } -func rowToString(row map[string]*dbutil.ColumnData) string { - var s strings.Builder - s.WriteString("{ ") - for key, val := range row { - if val.IsNull { - s.WriteString(fmt.Sprintf("%s: IsNull, ", key)) - } else { - s.WriteString(fmt.Sprintf("%s: %s, ", key, val.Data)) - } + colVals := make([][]byte, len(cols)) + colValsI := make([]interface{}, len(colVals)) + for i := range colValsI { + colValsI[i] = &colVals[i] } - s.WriteString(" }") - return s.String() -} + err = rows.Scan(colValsI...) + if err != nil { + return nil, errors.Trace(err) + } -func getPKValues(data map[string]*dbutil.ColumnData, cond *Cond) string { - var pkValues []string - for _, pkColumn := range cond.Table.PrimaryKey.Columns { - // TODO primary key cannot be null, if we uses unique key should make sure all columns are not null - pkValues = append(pkValues, string(data[pkColumn.Name.O].Data)) + result := make([]*dbutil.ColumnData, len(cols)) + for i := range colVals { + result[i] = &dbutil.ColumnData{ + Data: colVals[i], + IsNull: colVals[i] == nil, + } } - return strings.Join(pkValues, "-") + + return result, nil } -func NeedQuotes(tp byte) bool { - return !(dbutil.IsNumberType(tp) || dbutil.IsFloatType(tp)) +func getSourceRowsForCompare(rows []*rowChange) map[string][]*dbutil.ColumnData { + rowMap := make(map[string][]*dbutil.ColumnData, len(rows)) + for _, r := range rows { + colValues := make([]*dbutil.ColumnData, 0, len(r.data)) + for i := range r.data { + var colData []byte + if r.data[i] != nil { + // todo: may not right for some type, such as time related + colData = []byte(fmt.Sprintf("%v", r.data[i])) + } + colValues[i] = &dbutil.ColumnData{ + Data: colData, + IsNull: r.data[i] == nil, + } + } + rowMap[r.key] = colValues + } + return rowMap } diff --git a/dm/syncer/validator_cond.go b/dm/syncer/validator_cond.go index 128b1134d79..ffe9d5250d0 100644 --- a/dm/syncer/validator_cond.go +++ b/dm/syncer/validator_cond.go @@ -18,8 +18,9 @@ import ( ) type Cond struct { - Table *validateTableInfo - PkValues [][]string + Table *validateTableInfo + ColumnCnt int + PkValues [][]string } func (c *Cond) GetArgs() []interface{} { From c35dd4cc003c608d9665fd968f5b08c6a186e861 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Tue, 22 Feb 2022 17:36:07 +0800 Subject: [PATCH 14/36] genColData --- dm/syncer/validate_worker.go | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/dm/syncer/validate_worker.go b/dm/syncer/validate_worker.go index 6ac5aeb2748..c4961210e8d 100644 --- a/dm/syncer/validate_worker.go +++ b/dm/syncer/validate_worker.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb/parser/model" tidbmysql "github.com/pingcap/tidb/parser/mysql" + "github.com/shopspring/decimal" "go.uber.org/zap" tcontext "github.com/pingcap/tiflow/dm/pkg/context" @@ -269,6 +270,7 @@ func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.Column } return result, nil } + func ScanRow(rows *sql.Rows) ([]*dbutil.ColumnData, error) { cols, err := rows.Columns() if err != nil { @@ -304,8 +306,7 @@ func getSourceRowsForCompare(rows []*rowChange) map[string][]*dbutil.ColumnData for i := range r.data { var colData []byte if r.data[i] != nil { - // todo: may not right for some type, such as time related - colData = []byte(fmt.Sprintf("%v", r.data[i])) + colData = genColData(r.data[i]) } colValues[i] = &dbutil.ColumnData{ Data: colData, @@ -316,3 +317,16 @@ func getSourceRowsForCompare(rows []*rowChange) map[string][]*dbutil.ColumnData } return rowMap } + +func genColData(v interface{}) []byte { + switch dv := v.(type) { + case []byte: + return dv + case string: + return []byte(dv) + case decimal.Decimal: + return []byte(dv.String()) + } + s := fmt.Sprintf("%v", v) + return []byte(s) +} From e288c5330e860f318594a779090fc6a5a3fa31d7 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Tue, 22 Feb 2022 19:33:49 +0800 Subject: [PATCH 15/36] fix --- dm/syncer/data_validator.go | 3 ++- dm/syncer/validate_worker.go | 8 +++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 53e3ace07a4..0019cd1cf72 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -332,7 +332,7 @@ func (v *DataValidator) doValidate() { v.L.Info("start continuous validation") v.startValidateWorkers() - var currLoc binlog.Location + currLoc := location.CloneWithFlavor(v.cfg.Flavor) for { e, err := v.streamerController.GetEvent(v.tctx) if err != nil { @@ -411,6 +411,7 @@ func (v *DataValidator) startValidateWorkers() { v.wg.Add(v.workerCnt) for i := 0; i < v.workerCnt; i++ { worker := &validateWorker{ + cfg: v.cfg.ValidatorCfg, ctx: v.ctx, interval: v.validateInterval, validator: v, diff --git a/dm/syncer/validate_worker.go b/dm/syncer/validate_worker.go index c4961210e8d..30b7aeacfc4 100644 --- a/dm/syncer/validate_worker.go +++ b/dm/syncer/validate_worker.go @@ -29,6 +29,7 @@ import ( "github.com/shopspring/decimal" "go.uber.org/zap" + "github.com/pingcap/tiflow/dm/dm/config" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" @@ -36,6 +37,7 @@ import ( ) type validateWorker struct { + cfg config.ValidatorConfig ctx context.Context interval time.Duration validator *DataValidator @@ -262,8 +264,8 @@ func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.Column } pkVals := make([]string, 0, len(cond.Table.pkIndices)) for _, idx := range cond.Table.pkIndices { - colVal := fmt.Sprintf("%v", rowData[idx].Data) - pkVals = append(pkVals, colVal) + colVal := genColData(rowData[idx].Data) + pkVals = append(pkVals, string(colVal)) } pk := strings.Join(pkVals, "-") result[pk] = rowData @@ -302,7 +304,7 @@ func ScanRow(rows *sql.Rows) ([]*dbutil.ColumnData, error) { func getSourceRowsForCompare(rows []*rowChange) map[string][]*dbutil.ColumnData { rowMap := make(map[string][]*dbutil.ColumnData, len(rows)) for _, r := range rows { - colValues := make([]*dbutil.ColumnData, 0, len(r.data)) + colValues := make([]*dbutil.ColumnData, len(r.data)) for i := range r.data { var colData []byte if r.data[i] != nil { From 61821c9fcd8f6a108bd3c0ac9d81fbd9946178c4 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Tue, 22 Feb 2022 19:50:52 +0800 Subject: [PATCH 16/36] fix ut --- dm/syncer/validator_cond_test.go | 51 ++++++++++++++++---------------- 1 file changed, 25 insertions(+), 26 deletions(-) diff --git a/dm/syncer/validator_cond_test.go b/dm/syncer/validator_cond_test.go index 4361b59670d..87eee4c8c4b 100644 --- a/dm/syncer/validator_cond_test.go +++ b/dm/syncer/validator_cond_test.go @@ -17,28 +17,26 @@ import ( "database/sql" "fmt" "strconv" + "testing" - sqlmock "github.com/DATA-DOG/go-sqlmock" - . "github.com/pingcap/check" + "github.com/DATA-DOG/go-sqlmock" "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" + "github.com/stretchr/testify/require" ) -type testCondSuite struct{} - -var _ = Suite(&testCondSuite{}) - -func getTableDiff(c *C, db *sql.DB, schemaName, tableName, creatSQL string) *validateTableInfo { +func getTableDiff(t *testing.T, schemaName, tableName, creatSQL string) *validateTableInfo { var ( err error parser2 *parser.Parser tableInfo *model.TableInfo ) parser2 = parser.New() - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo, err = dbutil.GetTableInfoBySQL(creatSQL, parser2) - c.Assert(err, IsNil) + require.NoError(t, err) columnMap := make(map[string]*model.ColumnInfo) for _, col := range tableInfo.Columns { columnMap[col.Name.O] = col @@ -50,29 +48,30 @@ func getTableDiff(c *C, db *sql.DB, schemaName, tableName, creatSQL string) *val } } tableDiff := &validateTableInfo{ - Schema: schemaName, - Name: tableName, + Source: &filter.Table{ + Schema: schemaName, + Name: tableName, + }, Info: tableInfo, PrimaryKey: primaryIdx, - ColumnMap: columnMap, } return tableDiff } -func formatCond(c *C, db *sql.DB, schemaName, tblName, creatSQL string, pkvs [][]string) *Cond { - tblDiff := getTableDiff(c, db, schemaName, tblName, creatSQL) +func formatCond(t *testing.T, schemaName, tblName, creatSQL string, pkvs [][]string) *Cond { + tblDiff := getTableDiff(t, schemaName, tblName, creatSQL) return &Cond{ Table: tblDiff, PkValues: pkvs, } } -func (s *testCondSuite) TestCondSelectMultiKey(c *C) { +func TestCondSelectMultiKey(t *testing.T) { var ( res *sql.Rows ) db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) defer db.Close() creatTbl := "create table if not exists `test_cond`.`test1`(" + "a int," + @@ -87,7 +86,7 @@ func (s *testCondSuite) TestCondSelectMultiKey(c *C) { key1, key2 := strconv.Itoa(i+1), strconv.Itoa(i+2) pkValues = append(pkValues, []string{key1, key2}) } - cond := formatCond(c, db, "test_cond", "test1", creatTbl, pkValues) + cond := formatCond(t, "test_cond", "test1", creatTbl, pkValues) // format query string rowsQuery := fmt.Sprintf("SELECT COUNT(*) FROM %s WHERE %s;", "`test_cond`.`test1`", cond.GetWhere()) mock.ExpectQuery( @@ -95,20 +94,20 @@ func (s *testCondSuite) TestCondSelectMultiKey(c *C) { ).WithArgs( "1", "2", "2", "3", "3", "4", ).WillReturnRows(mock.NewRows([]string{"COUNT(*)"}).AddRow("3")) - c.Assert(err, IsNil) + require.NoError(t, err) res, err = db.Query(rowsQuery, cond.GetArgs()...) - c.Assert(err, IsNil) + require.NoError(t, err) var cnt int if res.Next() { err = res.Scan(&cnt) } - c.Assert(err, IsNil) - c.Assert(cnt, Equals, 3) + require.NoError(t, err) + require.Equal(t, 3, cnt) } -func (s *testCondSuite) TestCondGetWhereArgs(c *C) { +func TestCondGetWhereArgs(t *testing.T) { db, _, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) defer db.Close() type testCase struct { creatTbl string @@ -156,12 +155,12 @@ func (s *testCondSuite) TestCondGetWhereArgs(c *C) { }, } for i := 0; i < len(cases); i++ { - cond := formatCond(c, db, cases[i].schemaName, cases[i].tblName, cases[i].creatTbl, cases[i].pks) - c.Assert(cond.GetWhere(), Equals, cases[i].where) + cond := formatCond(t, cases[i].schemaName, cases[i].tblName, cases[i].creatTbl, cases[i].pks) + require.Equal(t, cases[i].where, cond.GetWhere()) rawArgs := cond.GetArgs() for j := 0; j < 3; j++ { curData := fmt.Sprintf("%v", rawArgs[j]) - c.Assert(curData, Equals, cases[i].args[j]) + require.Equal(t, cases[i].args[j], curData) } } } From f623835086b21a68c437c27873c57ee099a91308 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Wed, 23 Feb 2022 15:42:18 +0800 Subject: [PATCH 17/36] gen rowkey --- dm/syncer/data_validator.go | 15 +++++++-------- dm/syncer/validate_worker.go | 4 ++-- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 0019cd1cf72..5f2bfde45c0 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -435,12 +435,7 @@ func hashTablePk(s string) uint32 { } func (v *DataValidator) dispatchRowChange(key string, row *rowChange) { - // for every row in the table - // 1. join primary key by '-' - // 2. hash (tableName, primaryKey) to hashVal - // 3. dispatch the row change to dispatchMap[hashVal][tableName] - hashKey := strings.Join([]string{row.table.Source.String(), key}, "-") - hashVal := int(hashTablePk(hashKey)) % v.workerCnt + hashVal := int(hashTablePk(key)) % v.workerCnt v.workers[hashVal].rowChangeCh <- row } @@ -519,7 +514,7 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re for _, idx := range pkIndices { pkValue[idx] = fmt.Sprintf("%v", row[idx]) } - key := strings.Join(pkValue, "-") + key := genRowKey(pkValue) if changeType == rowUpdated { afterRowChangeType := changeType @@ -528,7 +523,7 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re for _, idx := range pkIndices { afterPkValue[idx] = fmt.Sprintf("%v", afterRow[idx]) } - afterKey := strings.Join(afterPkValue, "-") + afterKey := genRowKey(afterPkValue) if afterKey != key { // convert to delete and insert v.dispatchRowChange(key, &rowChange{ @@ -583,3 +578,7 @@ func getRowChangeType(t replication.EventType) rowChangeType { return rowDeleted } } + +func genRowKey(pkValues []string) string { + return strings.Join(pkValues, "-") +} diff --git a/dm/syncer/validate_worker.go b/dm/syncer/validate_worker.go index 30b7aeacfc4..6e451dff6b0 100644 --- a/dm/syncer/validate_worker.go +++ b/dm/syncer/validate_worker.go @@ -236,7 +236,7 @@ func (vw *validateWorker) compareData(sourceData, targetData []*dbutil.ColumnDat } func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.ColumnData, error) { - tctx := tcontext.NewContext(context.Background(), log.L()) + tctx := tcontext.NewContext(vw.ctx, log.L()) fullTableName := cond.Table.Target.String() pkColumnNames := make([]string, 0, len(cond.Table.pkIndices)) for i := range cond.Table.pkIndices { @@ -267,7 +267,7 @@ func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.Column colVal := genColData(rowData[idx].Data) pkVals = append(pkVals, string(colVal)) } - pk := strings.Join(pkVals, "-") + pk := genRowKey(pkVals) result[pk] = rowData } return result, nil From 64f94744c4ac3150e5f790f39396e7e81837d7c3 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Wed, 23 Feb 2022 15:45:51 +0800 Subject: [PATCH 18/36] gen rowkey --- dm/syncer/validate_worker.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dm/syncer/validate_worker.go b/dm/syncer/validate_worker.go index 6e451dff6b0..611ecacde5c 100644 --- a/dm/syncer/validate_worker.go +++ b/dm/syncer/validate_worker.go @@ -258,7 +258,7 @@ func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.Column result := make(map[string][]*dbutil.ColumnData, 0) for rows.Next() { - rowData, err := ScanRow(rows) + rowData, err := scanRow(rows) if err != nil { return nil, err } @@ -273,7 +273,7 @@ func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.Column return result, nil } -func ScanRow(rows *sql.Rows) ([]*dbutil.ColumnData, error) { +func scanRow(rows *sql.Rows) ([]*dbutil.ColumnData, error) { cols, err := rows.Columns() if err != nil { return nil, errors.Trace(err) From aa889e515b98709e7e7f00398ec475616ac3294e Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 24 Feb 2022 15:01:43 +0800 Subject: [PATCH 19/36] ut and fix --- dm/dm/config/task.go | 8 +- dm/pkg/conn/mockdb.go | 13 + dm/syncer/data_validator.go | 71 +-- dm/syncer/data_validator_test.go | 968 +++++++++++------------------- dm/syncer/validate_worker_test.go | 122 ++++ dm/syncer/validator_cond_test.go | 28 +- 6 files changed, 544 insertions(+), 666 deletions(-) create mode 100644 dm/syncer/validate_worker_test.go diff --git a/dm/dm/config/task.go b/dm/dm/config/task.go index 69f536236e0..5d661eeb894 100644 --- a/dm/dm/config/task.go +++ b/dm/dm/config/task.go @@ -60,6 +60,8 @@ const ( ValidationNone = "none" ValidationFast = "fast" ValidationFull = "full" + + DefaultValidatorWorkerCount = 4 ) // default config item values. @@ -337,7 +339,8 @@ func (m *SyncerConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { } type ValidatorConfig struct { - Mode string `yaml:"mode" toml:"mode" json:"mode"` + Mode string `yaml:"mode" toml:"mode" json:"mode"` + WorkerCount int `yaml:"worker-count" toml:"worker-count" json:"worker-count"` } func (v *ValidatorConfig) adjust() error { @@ -347,6 +350,9 @@ func (v *ValidatorConfig) adjust() error { if v.Mode != ValidationNone && v.Mode != ValidationFast && v.Mode != ValidationFull { return terror.ErrConfigValidationMode } + if v.WorkerCount <= 0 { + v.WorkerCount = DefaultValidatorWorkerCount + } return nil } diff --git a/dm/pkg/conn/mockdb.go b/dm/pkg/conn/mockdb.go index 8d8a98c409c..d5bd7f045ca 100644 --- a/dm/pkg/conn/mockdb.go +++ b/dm/pkg/conn/mockdb.go @@ -74,6 +74,19 @@ func InitVersionDB(c *check.C) sqlmock.Sqlmock { return mock } +func InitMockDBFull() (*sql.DB, sqlmock.Sqlmock, error) { + db, mock, err := sqlmock.New() + if err != nil { + return nil, nil, err + } + if mdbp, ok := DefaultDBProvider.(*mockDBProvider); ok { + mdbp.db = db + } else { + DefaultDBProvider = &mockDBProvider{db: db} + } + return db, mock, err +} + // TODO: export Config in https://github.com/pingcap/tidb/blob/a8fa29b56d633b1ec843e21cb89131dd4fd601db/br/pkg/mock/mock_cluster.go#L35 // Cluster is mock tidb cluster. type Cluster struct { diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 5f2bfde45c0..d1c2e77bb99 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser/model" + "go.uber.org/atomic" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/config" @@ -115,12 +116,10 @@ type DataValidator struct { streamerController *StreamerController result pb.ProcessResult - batchRowCnt int validateInterval time.Duration workers []*validateWorker rowsEventChan chan *replication.BinlogEvent // unbuffered is enough - changeEventCount []int - tables map[string]*validateTableInfo + changeEventCount []atomic.Int64 workerCnt int // such as table without primary key @@ -130,27 +129,31 @@ type DataValidator struct { func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *DataValidator { v := &DataValidator{ - cfg: cfg, - syncer: syncerObj, - stage: pb.Stage_Stopped, + cfg: cfg, + syncer: syncerObj, + stage: pb.Stage_Stopped, + // todo: many place may put into this channel, choose a proper channel size or enhance error handling errChan: make(chan error, 1), } v.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) - v.workerCnt = defaultWorkerCnt - v.batchRowCnt = defaultBatchRowCount + v.workerCnt = cfg.ValidatorCfg.WorkerCount v.workers = make([]*validateWorker, v.workerCnt) v.rowsEventChan = make(chan *replication.BinlogEvent) - v.tables = make(map[string]*validateTableInfo) - v.changeEventCount = make([]int, 4) + v.changeEventCount = make([]atomic.Int64, 4) v.validateInterval = validationInterval + v.unsupportedTable = make(map[string]string) v.waitSyncerTimer = utils.NewStoppedTimer() return v } func (v *DataValidator) initialize() error { + v.ctx, v.cancel = context.WithCancel(context.Background()) + v.tctx = tcontext.NewContext(v.ctx, v.L) + v.result.Reset() + newCtx, cancelFunc := context.WithTimeout(v.ctx, unit.DefaultInitTimeout) defer cancelFunc() tctx := tcontext.NewContext(newCtx, v.L) @@ -162,6 +165,7 @@ func (v *DataValidator) initialize() error { } dbconn.CloseBaseDB(tctx, v.fromDB) dbconn.CloseBaseDB(tctx, v.toDB) + v.cancel() }() dbCfg := v.cfg.From @@ -199,9 +203,6 @@ func (v *DataValidator) Start(expect pb.Stage) { return } - v.ctx, v.cancel = context.WithCancel(context.Background()) - v.tctx = tcontext.NewContext(v.ctx, v.L) - if err := v.initialize(); err != nil { v.fillResult(err, false) return @@ -229,23 +230,20 @@ func (v *DataValidator) fillResult(err error, needLock bool) { defer v.Unlock() } - var errs []*pb.ProcessError - if utils.IsContextCanceledError(err) { - v.L.Info("filter out context cancelled error", log.ShortError(err)) - } else { - errs = append(errs, unit.NewProcessError(err)) - } - + // todo: if error, validation is stopped( and cancelled), and we may receive a cancelled error. + // todo: do we need this cancel field? isCanceled := false select { case <-v.ctx.Done(): isCanceled = true default: } + v.result.IsCanceled = isCanceled - v.result = pb.ProcessResult{ - IsCanceled: isCanceled, - Errors: errs, + if utils.IsContextCanceledError(err) { + v.L.Info("filter out context cancelled error", log.ShortError(err)) + } else { + v.result.Errors = append(v.result.Errors, unit.NewProcessError(err)) } } @@ -302,6 +300,9 @@ func (v *DataValidator) waitSyncerSynced(currLoc binlog.Location) error { } func (v *DataValidator) waitSyncerRunning() error { + if v.syncer.IsRunning() { + return nil + } for { select { case <-v.ctx.Done(): @@ -323,10 +324,12 @@ func (v *DataValidator) doValidate() { // todo: syncer may change replication location(start from timestamp, sharding resync), how validator react? location := v.syncer.checkpoint.FlushedGlobalPoint() - err := v.streamerController.Start(v.tctx, location) - if err != nil { - v.errChan <- terror.Annotate(err, "fail to start streamer controller") - return + if v.streamerController.IsClosed() { + err := v.streamerController.Start(v.tctx, location) + if err != nil { + v.errChan <- terror.Annotate(err, "fail to start streamer controller") + return + } } v.L.Info("start continuous validation") @@ -364,7 +367,7 @@ func (v *DataValidator) doValidate() { switch ev := e.Event.(type) { case *replication.RowsEvent: - if err = v.processEventRows(e.Header, ev); err != nil { + if err = v.processRowsEvent(e.Header, ev); err != nil { v.L.Warn("failed to process event: ", zap.Reflect("error", err)) v.errChan <- terror.Annotate(err, "failed to process event") return @@ -439,7 +442,7 @@ func (v *DataValidator) dispatchRowChange(key string, row *rowChange) { v.workers[hashVal].rowChangeCh <- row } -func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *replication.RowsEvent) error { +func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *replication.RowsEvent) error { sourceTable := &filter.Table{ Schema: string(ev.Table.Schema), Name: string(ev.Table.Table), @@ -491,7 +494,7 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re } changeType := getRowChangeType(header.EventType) - v.changeEventCount[changeType]++ + v.changeEventCount[changeType].Inc() columnMap := make(map[string]*model.ColumnInfo) for _, col := range tableInfo.Columns { @@ -558,14 +561,6 @@ func (v *DataValidator) processEventRows(header *replication.EventHeader, ev *re return nil } -func (v *DataValidator) getRowCount(c map[string]*tableChange) int { - res := 0 - for _, val := range c { - res += len(val.rows) - } - return res -} - // getRowChangeType should be called only when the event type is RowsEvent func getRowChangeType(t replication.EventType) rowChangeType { switch t { diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index 149dcdfbf31..1b125baf134 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -16,698 +16,430 @@ package syncer import ( "context" "database/sql" - "database/sql/driver" - "fmt" - "os" - "sort" + "testing" "time" - "github.com/pingcap/tiflow/dm/pkg/log" - "github.com/DATA-DOG/go-sqlmock" "github.com/go-mysql-org/go-mysql/mysql" + gmysql "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" - . "github.com/pingcap/check" - "github.com/pingcap/tidb-tools/pkg/dbutil" - parsermysql "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/errors" + "github.com/pingcap/tidb-tools/pkg/filter" + "github.com/stretchr/testify/require" + "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/tiflow/dm/pkg/binlog" "github.com/pingcap/tiflow/dm/pkg/binlog/event" "github.com/pingcap/tiflow/dm/pkg/conn" "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/router" + "github.com/pingcap/tiflow/dm/pkg/schema" + "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/syncer/dbconn" ) -type testDataValidatorSuite struct { - eventsGenerator *event.Generator - cfg *config.SubTaskConfig -} - -var _ = Suite(&testDataValidatorSuite{}) - -func (d *testDataValidatorSuite) SetUpSuite(c *C) { - previousGTIDSetStr := "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-14,406a3f61-690d-11e7-87c5-6c92bf46f384:1-94321383" +func genEventGenerator(t *testing.T) *event.Generator { + previousGTIDSetStr := "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-14" previousGTIDSet, err := gtid.ParserGTID(mysql.MySQLFlavor, previousGTIDSetStr) - if err != nil { - c.Fatal(err) - } + require.NoError(t, err) latestGTIDStr := "3ccc475b-2343-11e7-be21-6c0b84d59f30:14" latestGTID, err := gtid.ParserGTID(mysql.MySQLFlavor, latestGTIDStr) - c.Assert(err, IsNil) - d.eventsGenerator, err = event.NewGenerator(mysql.MySQLFlavor, 1, 0, latestGTID, previousGTIDSet, 0) - if err != nil { - c.Fatal(err) - } - loaderDir, err := os.MkdirTemp("", "loader") - c.Assert(err, IsNil) + require.NoError(t, err) + eventsGenerator, err := event.NewGenerator(mysql.MySQLFlavor, 1, 0, latestGTID, previousGTIDSet, 0) + require.NoError(t, err) + require.NoError(t, err) + + return eventsGenerator +} + +func genSubtaskConfig(t *testing.T) *config.SubTaskConfig { loaderCfg := config.LoaderConfig{ - Dir: loaderDir, + Dir: t.TempDir(), } - d.cfg = &config.SubTaskConfig{ + cfg := &config.SubTaskConfig{ From: config.GetDBConfigForTest(), To: config.GetDBConfigForTest(), + Timezone: "UTC", ServerID: 101, - MetaSchema: "test", Name: "validator_ut", ShadowTableRules: []string{config.DefaultShadowTableRules}, TrashTableRules: []string{config.DefaultTrashTableRules}, Mode: config.ModeIncrement, - Flavor: "mysql", + Flavor: mysql.MySQLFlavor, LoaderConfig: loaderCfg, + SyncerConfig: config.SyncerConfig{ + EnableGTID: false, + }, + ValidatorCfg: config.ValidatorConfig{ + Mode: config.ModeFull, + WorkerCount: 1, + }, } - d.cfg.Experimental.AsyncCheckpointFlush = true - d.cfg.From.Adjust() - d.cfg.To.Adjust() + cfg.Experimental.AsyncCheckpointFlush = true + cfg.From.Adjust() + cfg.To.Adjust() + + cfg.UseRelay = false - d.cfg.UseRelay = false - c.Assert(log.InitLogger(&log.Config{}), IsNil) + return cfg } -func (d *testDataValidatorSuite) genDBConn(c *C, db *sql.DB) *dbconn.DBConn { +func genDBConn(t *testing.T, db *sql.DB, cfg *config.SubTaskConfig) *dbconn.DBConn { baseDB := conn.NewBaseDB(db, func() {}) baseConn, err := baseDB.GetBaseConn(context.Background()) - c.Assert(err, IsNil) - cfg, err := d.cfg.Clone() - c.Assert(err, IsNil) + require.NoError(t, err) return &dbconn.DBConn{ BaseConn: baseConn, Cfg: cfg, } } -func (d *testDataValidatorSuite) TestRowDataIteratorImpl(c *C) { - var ( - iter RowDataIterator - dbConn *dbconn.DBConn - ) - db, mock, err := sqlmock.New() - c.Assert(err, IsNil) - type testTableSchm struct { - a string - b string - } - rowVals := []testTableSchm{ - { - a: "a", - b: "100", - }, - { - a: "c", - b: "200", - }, - { - a: "d", - b: "300", - }, - { - a: "e", - b: "400", - }, - } - expectRows := sqlmock.NewRows([]string{"a", "b"}) - for i := 0; i < len(rowVals); i++ { - expectRows = expectRows.AddRow(rowVals[i].a, rowVals[i].b) +func Test_validator_StartStop(t *testing.T) { + cfg := genSubtaskConfig(t) + syncerObj := NewSyncer(cfg, nil, nil) + + // validator already running + validator := NewContinuousDataValidator(cfg, syncerObj) + validator.stage = pb.Stage_Running + validator.Start(pb.Stage_InvalidStage) + // if validator already running, Start will return immediately, so we check validator.ctx which has not initialized. + require.Nil(t, validator.ctx) + + // failed to init + cfg.From = config.DBConfig{ + Host: "invalid host", + Port: 3306, + User: "root", } - mock.ExpectQuery( - "SELECT .* FROM .* WHERE a in .*", - ).WithArgs( - "a", "c", "d", "e", - ).WillReturnRows(expectRows) - c.Assert(err, IsNil) - cond := formatCond(c, db, "test_row", "test", ` - create table if not exists test_row.test( - a char(1), - b int, - primary key(a) - ); - `, [][]string{{"a"}, {"c"}, {"d"}, {"e"}}) - c.Assert(err, IsNil) - dbConn = d.genDBConn(c, db) - iter, err = getRowsFrom(cond, dbConn) - c.Assert(err, IsNil) - for i := range rowVals { - curVal, err := iter.Next() - c.Assert(err, IsNil) - keys := make([]string, 0) - for key := range curVal { - keys = append(keys, key) - } - sort.Slice(keys, func(i, j int) bool { - return keys[i] < keys[j] - }) - c.Assert(keys[0], Equals, "a") - c.Assert(keys[1], Equals, "b") - c.Assert(string(curVal["a"].Data), Equals, rowVals[i].a) - c.Assert(string(curVal["b"].Data), Equals, rowVals[i].b) + validator = NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + require.Equal(t, pb.Stage_Stopped, validator.Stage()) + require.Len(t, validator.result.Errors, 1) + + // start with Stopped stage + _, _, err := conn.InitMockDBFull() + require.NoError(t, err) + defer func() { + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + }() + validator = NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + require.Equal(t, pb.Stage_Stopped, validator.Stage()) + require.Len(t, validator.result.Errors, 0) + + // normal start & stop + validator = NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Running) + require.Equal(t, pb.Stage_Running, validator.Stage()) + require.True(t, validator.Started()) + validator.Stop() + require.Equal(t, pb.Stage_Stopped, validator.Stage()) + + // stop before start, should not panic + validator = NewContinuousDataValidator(cfg, syncerObj) + validator.Stop() +} + +func Test_validator_fillResult(t *testing.T) { + cfg := genSubtaskConfig(t) + syncerObj := NewSyncer(cfg, nil, nil) + _, _, err := conn.InitMockDBFull() + require.NoError(t, err) + defer func() { + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + }() + + validator := NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Running) + validator.fillResult(errors.New("test error"), false) + require.Len(t, validator.result.Errors, 1) + validator.fillResult(errors.New("test error"), true) + require.Len(t, validator.result.Errors, 2) + validator.Stop() + validator.fillResult(validator.ctx.Err(), true) + require.Len(t, validator.result.Errors, 2) +} + +func Test_validator_errorProcessRoutine(t *testing.T) { + cfg := genSubtaskConfig(t) + syncerObj := NewSyncer(cfg, nil, nil) + _, _, err := conn.InitMockDBFull() + require.NoError(t, err) + defer func() { + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + }() + + validator := NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Running) + require.Equal(t, pb.Stage_Running, validator.Stage()) + validator.errChan <- errors.New("test error") + require.True(t, utils.WaitSomething(20, 100*time.Millisecond, func() bool { + return validator.Stage() == pb.Stage_Stopped + })) + require.Len(t, validator.result.Errors, 1) +} + +type mockedCheckPointForValidator struct { + CheckPoint + cnt int + currLoc binlog.Location + nextLoc binlog.Location +} + +func (c *mockedCheckPointForValidator) FlushedGlobalPoint() binlog.Location { + c.cnt++ + if c.cnt <= 2 { + return c.currLoc + } else { + return c.nextLoc } - nxtVal, err := iter.Next() - c.Assert(nxtVal, IsNil) - c.Assert(err, IsNil) } -func (r *testDataValidatorSuite) TestRowChangeIteratorImpl(c *C) { - db, _, err := sqlmock.New() - c.Assert(err, IsNil) - rows := []*rowChange{ - { - pkValues: []string{"pk", "1"}, - data: []interface{}{"pk", "1", "some data"}, - theType: rowInsert, - }, - { - pkValues: []string{"pk", "2"}, - data: []interface{}{"pk", "2", "some data"}, - theType: rowDeleted, - }, - { - pkValues: []string{"pkg", "2"}, - data: []interface{}{"pkg", "2", "some data"}, - theType: rowDeleted, - }, - { - pkValues: []string{"pke", "2"}, - data: []interface{}{"pke", "2", "some data"}, - theType: rowUpdated, - }, +func Test_validator_waitSyncerSynced(t *testing.T) { + cfg := genSubtaskConfig(t) + syncerObj := NewSyncer(cfg, nil, nil) + _, _, err := conn.InitMockDBFull() + require.NoError(t, err) + defer func() { + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + }() + + currLoc := binlog.NewLocation(cfg.Flavor) + validator := NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + require.NoError(t, validator.waitSyncerSynced(currLoc)) + + // cancelled + currLoc.Position = gmysql.Position{ + Name: "mysql-bin.000001", + Pos: 100, } - testCases := []map[string]string{ - { - "pk1": "pk", - "pk2": "1", - "other": "some data", - }, - { - "pk1": "pk", - "pk2": "2", - "other": "some data", - }, - { - "pk1": "pke", - "pk2": "2", - "other": "some data", - }, - { - "pk1": "pkg", - "pk2": "2", - "other": "some data", - }, + validator = NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + validator.cancel() + require.ErrorIs(t, validator.waitSyncerSynced(currLoc), context.Canceled) + + currLoc.Position = gmysql.Position{ + Name: "mysql-bin.000001", + Pos: 100, } - creatTbl := `create table if not exists test_it.test1 ( - pk1 varchar(4), - pk2 int, - other text, - primary key(pk1, pk2) - );` - tableDiff := getTableDiff(c, db, "test_it", "test1", creatTbl) - var iter RowDataIterator - iter, err = getSourceRowsForCompare(rows) - c.Assert(err, IsNil) - for i := 0; i < len(rows); i++ { - res, err := iter.Next() - c.Assert(err, IsNil) - c.Assert(res, NotNil) - colNames := []string{} - for key := range res { - colNames = append(colNames, key) - c.Assert(string(res[key].Data), Equals, testCases[i][key]) - } - sort.Slice(colNames, func(left, right int) bool { - return colNames[left] < colNames[right] - }) - c.Assert(colNames[0], Equals, "other") - c.Assert(colNames[1], Equals, "pk1") - c.Assert(colNames[2], Equals, "pk2") + syncerObj.checkpoint = &mockedCheckPointForValidator{ + currLoc: binlog.NewLocation(cfg.Flavor), + nextLoc: currLoc, } + validator = NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + require.NoError(t, validator.waitSyncerSynced(currLoc)) +} + +func Test_validator_waitSyncerRunning(t *testing.T) { + cfg := genSubtaskConfig(t) + syncerObj := NewSyncer(cfg, nil, nil) + _, _, err := conn.InitMockDBFull() + require.NoError(t, err) + defer func() { + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + }() + + validator := NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + validator.cancel() + require.Error(t, validator.waitSyncerRunning()) + + validator = NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + syncerObj.stage.Store(int32(pb.Stage_Running)) + require.NoError(t, validator.waitSyncerRunning()) + + validator = NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + syncerObj.stage.Store(int32(pb.Stage_Stopped)) + go func() { + time.Sleep(3 * time.Second) + syncerObj.stage.Store(int32(pb.Stage_Running)) + }() + require.NoError(t, validator.waitSyncerRunning()) } -func (d *testDataValidatorSuite) TestGetRowsFrom(c *C) { - type testCase struct { - schemaName string - tblName string - creatSQL string - pkValues [][]string - allCols []string - rowData [][]string - querySQL string +func Test_validator_doValidate(t *testing.T) { + var ( + schemaName = "test" + tableName = "tbl" + tableName2 = "tbl2" + tableName3 = "tbl3" + tableName4 = "tbl4" + createTableSql = "CREATE TABLE `" + tableName + "`(id int primary key, v varchar(100))" + createTableSql2 = "CREATE TABLE `" + tableName2 + "`(id int primary key)" + createTableSql3 = "CREATE TABLE `" + tableName3 + "`(id int, v varchar(100))" + ) + cfg := genSubtaskConfig(t) + _, _, err := conn.InitMockDBFull() + require.NoError(t, err) + defer func() { + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + }() + + syncerObj := NewSyncer(cfg, nil, nil) + syncerObj.stage.Store(int32(pb.Stage_Running)) + syncerObj.tableRouter, err = router.NewRouter(cfg.CaseSensitive, []*router.TableRule{}) + require.NoError(t, err) + currLoc := binlog.NewLocation(cfg.Flavor) + currLoc.Position = gmysql.Position{ + Name: "mysql-bin.000001", + Pos: 3000, } - testCases := []testCase{ - { - schemaName: "test1", - tblName: "tbl1", - creatSQL: `create table if not exists test1.tbl1( - a int, - b int, - c int, - primary key(a, b) - );`, - pkValues: [][]string{ - {"1", "2"}, {"3", "4"}, {"5", "6"}, - }, - allCols: []string{"a", "b", "c"}, - rowData: [][]string{ - {"1", "2", "3"}, {"3", "4", "5"}, {"5", "6", "7"}, - }, - querySQL: "SELECT .* FROM .*test1.*", - }, - { - schemaName: "test2", - tblName: "tbl2", - creatSQL: `create table if not exists test2.tbl2( - a char(10), - other text, - primary key(a) - );`, - pkValues: [][]string{ - {"a"}, {"b"}, {"c"}, - }, - allCols: []string{"a", "other"}, - rowData: [][]string{ - {"a", "some data"}, {"b", "some data"}, {"c", "some data"}, - }, - querySQL: "SELECT .* FROM .*test2.*", - }, + syncerObj.checkpoint = &mockedCheckPointForValidator{ + currLoc: binlog.NewLocation(cfg.Flavor), + nextLoc: currLoc, + cnt: 2, } db, mock, err := sqlmock.New() - c.Assert(err, IsNil) - for i, testCase := range testCases { - args := []driver.Value{} - for _, arr := range testCases[i].pkValues { - for _, val := range arr { - args = append(args, val) - } - } - dataRows := mock.NewRows(testCase.allCols) - for j := range testCases[i].rowData { - args := []driver.Value{} - for _, val := range testCase.rowData[j] { - args = append(args, val) - } - dataRows = dataRows.AddRow(args...) - } - mock.ExpectQuery(testCase.querySQL).WithArgs(args...).WillReturnRows(dataRows) - cond := formatCond( - c, db, - testCase.schemaName, - testCase.tblName, - testCase.creatSQL, - testCase.pkValues, - ) - dbConn := d.genDBConn(c, db) - var iter RowDataIterator - iter, err = getRowsFrom(cond, dbConn) - c.Assert(err, IsNil) - var res map[string]*dbutil.ColumnData - for j := 0; j < 3; j++ { - res, err = iter.Next() - c.Assert(err, IsNil) - pkRes := getPKValues(res, cond) - // for each primary key - for k, val := range testCase.pkValues[j] { - c.Assert(pkRes[k], Equals, val) - } - // for each col - for k, val := range testCase.rowData[j] { - colName := testCase.allCols[k] - c.Assert(string(res[colName].Data), Equals, val) - } - } - res, err = iter.Next() - c.Assert(res, IsNil) - c.Assert(err, IsNil) - } -} + require.NoError(t, err) + //mock.ExpectQuery("SHOW CREATE TABLE.*").WillReturnRows( + // sqlmock.NewRows([]string{"Table", "Create Table"}). + // AddRow(tableName, createTableSql). + // AddRow(tableName2, createTableSql2), + //) + mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows( + mock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", ""), + ) + dbConn, err := db.Conn(context.Background()) + require.NoError(t, err) + syncerObj.downstreamTrackConn = &dbconn.DBConn{Cfg: cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} + syncerObj.schemaTracker, err = schema.NewTracker(context.Background(), cfg.Name, defaultTestSessionCfg, syncerObj.downstreamTrackConn) + require.NoError(t, syncerObj.schemaTracker.CreateSchemaIfNotExists(schemaName)) + require.NoError(t, syncerObj.schemaTracker.Exec(context.Background(), schemaName, createTableSql)) + require.NoError(t, syncerObj.schemaTracker.Exec(context.Background(), schemaName, createTableSql2)) + require.NoError(t, syncerObj.schemaTracker.Exec(context.Background(), schemaName, createTableSql3)) -func (d *testDataValidatorSuite) TestRowToString(c *C) { - testCases := []map[string]*dbutil.ColumnData{ + generator := genEventGenerator(t) + rotateEvent, _, err := generator.Rotate("mysql-bin.000001", 0) + require.NoError(t, err) + insertData := []*event.DMLData{ { - "pk-1": &dbutil.ColumnData{ - Data: []byte("some data"), - IsNull: false, - }, - "pk-2": &dbutil.ColumnData{ - Data: []byte("simple data"), - IsNull: false, + TableID: 11, + Schema: schemaName, + Table: tableName, + ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + Rows: [][]interface{}{ + {int32(1), "a"}, + {int32(2), "b"}, + {int32(3), "c"}, }, }, + // 2 columns in binlog, but ddl of tbl2 only has one column { - "pk-3": &dbutil.ColumnData{ - Data: nil, - IsNull: true, - }, - "pk-4": &dbutil.ColumnData{ - Data: []byte("data"), - IsNull: false, + TableID: 12, + Schema: schemaName, + Table: tableName2, + ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + Rows: [][]interface{}{ + {int32(1), "a"}, + {int32(2), "b"}, + {int32(3), "c"}, }, }, - } - expectedStr := []string{ - "({ pk-1: some data, pk-2: simple data, }|{ pk-2: simple data, pk-1: some data, })", - "({ pk-3: IsNull, pk-4: data, }|{ pk-4: data, pk-3: IsNull, })", - } - for i, testCase := range testCases { - ret := rowToString(testCase) - c.Assert(ret, Matches, expectedStr[i]) - } -} - -func (d *testDataValidatorSuite) TestNeedQuotes(c *C) { - testCases := map[byte]bool{ - parsermysql.TypeTiny: false, - parsermysql.TypeShort: false, - parsermysql.TypeLong: false, - parsermysql.TypeLonglong: false, - parsermysql.TypeInt24: false, - parsermysql.TypeYear: false, - parsermysql.TypeFloat: false, - parsermysql.TypeDouble: false, - parsermysql.TypeNewDecimal: false, - parsermysql.TypeDuration: true, - parsermysql.TypeDatetime: true, - parsermysql.TypeNewDate: true, - parsermysql.TypeVarchar: true, - parsermysql.TypeBlob: true, - parsermysql.TypeVarString: true, - parsermysql.TypeString: true, - } - for typ, val := range testCases { - c.Assert(NeedQuotes(typ), Equals, val) - } -} - -func (d *testDataValidatorSuite) TestGetPKValues(c *C) { - testCases := []map[string]*dbutil.ColumnData{ + // tbl3 has no primary key { - "col1": &dbutil.ColumnData{ - Data: []byte("some data"), - }, - "col2": &dbutil.ColumnData{ - Data: []byte("data"), - }, - "col3": &dbutil.ColumnData{ - IsNull: true, + TableID: 13, + Schema: schemaName, + Table: tableName3, + ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + Rows: [][]interface{}{ + {int32(1), "a"}, + {int32(2), "b"}, + {int32(3), "c"}, }, }, + // tbl3 has no primary key, since we met it before, will return immediately { - "c1": &dbutil.ColumnData{ - Data: []byte("lk"), - }, - "c2": &dbutil.ColumnData{ - Data: []byte("1001"), + TableID: 13, + Schema: schemaName, + Table: tableName3, + ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + Rows: [][]interface{}{ + {int32(4), "a"}, }, }, } - expectedOut := [][]string{ - {"some data"}, - {"lk", "1001"}, - } - type testTable struct { - schemaName string - tableName string - creatSQL string - } - testTables := []testTable{ + updateData := []*event.DMLData{ { - schemaName: "test1", - tableName: "tbl1", - creatSQL: `create table if not exists test1.tbl1( - col1 varchar(10), - col2 text, - col3 varchar(20), - primary key(col1) - );`, - }, - { - schemaName: "test2", - tableName: "tbl2", - creatSQL: `create table if not exists test2.tbl2( - c1 char(2), - c2 int, - primary key(c1, c2) - );`, + TableID: 11, + Schema: schemaName, + Table: tableName, + ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + Rows: [][]interface{}{ + // update non-primary column + {int32(3), "c"}, + {int32(3), "d"}, + // update primary column and non-primary column + {int32(1), "a"}, + {int32(4), "b"}, + }, }, } - db, _, err := sqlmock.New() - c.Assert(err, IsNil) - for i := range testCases { - cond := formatCond( - c, - db, - testTables[i].schemaName, - testTables[i].tableName, - testTables[i].creatSQL, - [][]string{}, - ) - ret := getPKValues(testCases[i], cond) - for j, val := range expectedOut[i] { - c.Assert(ret[j], Equals, val) - } - } -} - -func (d *testDataValidatorSuite) generateEvents(binlogEvents mockBinlogEvents, c *C) []*replication.BinlogEvent { - events := make([]*replication.BinlogEvent, 0, 1024) - for _, e := range binlogEvents { - switch e.typ { - case DBCreate: - evs, _, err := d.eventsGenerator.GenCreateDatabaseEvents(e.args[0].(string)) - c.Assert(err, IsNil) - events = append(events, evs...) - case DBDrop: - evs, _, err := d.eventsGenerator.GenDropDatabaseEvents(e.args[0].(string)) - c.Assert(err, IsNil) - events = append(events, evs...) - case TableCreate: - evs, _, err := d.eventsGenerator.GenCreateTableEvents(e.args[0].(string), e.args[1].(string)) - c.Assert(err, IsNil) - events = append(events, evs...) - case TableDrop: - evs, _, err := d.eventsGenerator.GenDropTableEvents(e.args[0].(string), e.args[1].(string)) - c.Assert(err, IsNil) - events = append(events, evs...) - - case DDL: - evs, _, err := d.eventsGenerator.GenDDLEvents(e.args[0].(string), e.args[1].(string), 0) - c.Assert(err, IsNil) - events = append(events, evs...) - - case Write, Update, Delete: - dmlData := []*event.DMLData{ - { - TableID: e.args[0].(uint64), - Schema: e.args[1].(string), - Table: e.args[2].(string), - ColumnType: e.args[3].([]byte), - Rows: e.args[4].([][]interface{}), - }, - } - var eventType replication.EventType - switch e.typ { - case Write: - eventType = replication.WRITE_ROWS_EVENTv2 - case Update: - eventType = replication.UPDATE_ROWS_EVENTv2 - case Delete: - eventType = replication.DELETE_ROWS_EVENTv2 - default: - c.Fatal(fmt.Sprintf("mock event generator don't support event type: %d", e.typ)) - } - evs, _, err := d.eventsGenerator.GenDMLEvents(eventType, dmlData, 0) - c.Assert(err, IsNil) - events = append(events, evs...) - } - } - return events -} - -func (d *testDataValidatorSuite) TestDoValidate(c *C) { - type testCase struct { - schemaName string - tblName string - creatSQL string - binlogEvs []mockBinlogEvent - selectSQLs []string // mock in toDB - retRows [][][]string - colNames []string - failRowCnt int - failRowPKs []string - } - batchSize := 2 - testCases := []testCase{ + deleteData := []*event.DMLData{ { - schemaName: "test1", - tblName: "tbl1", - creatSQL: `create table if not exists test1.tbl1( - a int, - b int, - c text, - primary key(a, b));`, - binlogEvs: []mockBinlogEvent{ - {typ: Write, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(1), int32(2), "some data1"}}}}, - {typ: Write, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(3), int32(4), "some data2"}}}}, - - {typ: Update, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(1), int32(2), "some data1"}, {int32(1), int32(3), "some data3"}}}}, - {typ: Delete, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(3), int32(4), "some data2"}}}}, - - {typ: Write, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(5), int32(6), "some data4"}}}}, - {typ: Update, args: []interface{}{uint64(8), "test1", "tbl1", []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, [][]interface{}{{int32(5), int32(6), "some data4"}, {int32(5), int32(7), "some data4"}}}}, + TableID: 11, + Schema: schemaName, + Table: tableName, + ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + Rows: [][]interface{}{ + {int32(3), "c"}, }, - selectSQLs: []string{ - "SELECT .* FROM .*test1.* WHERE .*a,b.* in .*", // batch1: insert row1, row2 - "SELECT .* FROM .*test1.* WHERE .*a,b.* in .*", // batch2: update row1 - "SELECT .* FROM .*test1.* WHERE .*a,b.* in .*", // batch2: delete row2 - "SELECT .* FROM .*test1.* WHERE .*a,b.* in .*", // batch3: insert and update row3, only query (5, 7) - "SELECT .* FROM .*test1.* WHERE .*a,b.* in .*", // batch3: update query (5, 6) - }, - retRows: [][][]string{ - { - {"1", "2", "some data1"}, // insert - {"3", "4", "some data2"}, - }, - { - {"1", "3", "some data3"}, // update - }, - { - {}, // delete - }, - { - {"5", "7", "some data4"}, - }, - { - {}, - }, - }, - colNames: []string{"a", "b", "c"}, - failRowCnt: 0, - failRowPKs: []string{}, }, + // no ddl for this table { - // stale read in downstream and got erronous result - // but row2's primary key is reused and inserted again - // the error is restored - schemaName: "test2", - tblName: "tbl2", - creatSQL: `create table if not exists test2.tbl2( - a varchar(10), - b int, - c float, - primary key(a));`, - binlogEvs: []mockBinlogEvent{ - {typ: Write, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val1", int32(1), float32(1.2)}}}}, - {typ: Write, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val2", int32(2), float32(2.2)}}}}, - - {typ: Delete, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val1", int32(1), float32(1.2)}}}}, - {typ: Delete, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val2", int32(2), float32(2.2)}}}}, - - {typ: Write, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val3", int32(3), float32(3.2)}}}}, - {typ: Write, args: []interface{}{uint64(8), "test2", "tbl2", []byte{mysql.MYSQL_TYPE_STRING, mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_FLOAT}, [][]interface{}{{"val2", int32(2), float32(4.2)}}}}, - }, - selectSQLs: []string{ - "SELECT .* FROM .*test2.* WHERE .*a.* in.*", // batch1: query row1 and row2 - "SELECT .* FROM .*test2.* WHERE .*a.* in.*", // batch2: query row1 and row2 - "SELECT .* FROM .*test2.* WHERE .*a.* in.*", // batch3: query row2 and row3 - }, - retRows: [][][]string{ - { - {"val1", "1", "1.2"}, - {"val2", "2", "2.2"}, - }, - { - {"val1", "1", "1.2"}, - {"val2", "2", "2.2"}, - }, - { - {"val2", "2", "4.2"}, - {"val3", "3", "3.2"}, - }, + TableID: 14, + Schema: schemaName, + Table: tableName4, + ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + Rows: [][]interface{}{ + {int32(4), "c"}, }, - colNames: []string{"a", "b", "c"}, - failRowCnt: 1, - failRowPKs: []string{"val1"}, }, } - for _, testCase := range testCases { - var ( - fromDB, toDB *sql.DB - fromMock, toMock sqlmock.Sqlmock - err error - ) - fromDB, fromMock, err = sqlmock.New() - c.Assert(err, IsNil) - toDB, toMock, err = sqlmock.New() - c.Assert(err, IsNil) - syncerObj := NewSyncer(d.cfg, nil, nil) - c.Assert(syncerObj, NotNil) - validator := NewContinuousDataValidator(d.cfg, syncerObj) - validator.Start(pb.Stage_Paused) // init but will return soon - validator.result = pb.ProcessResult{} // clear error - validator.workerCnt = 1 - validator.validateInterval = 100 * time.Second // never retry - events1 := testCase.binlogEvs - mockStreamerProducer := &MockStreamProducer{d.generateEvents(events1, c)} - mockStreamer, err := mockStreamerProducer.generateStreamer(binlog.NewLocation("")) - c.Assert(err, IsNil) - validator.streamerController = &StreamerController{ - streamerProducer: mockStreamerProducer, - streamer: mockStreamer, - closed: false, - } - // validate every 2 rows updated - validator.batchRowCnt = batchSize - validator.fromDB = conn.NewBaseDB(fromDB, func() {}) - validator.fromDBConn = d.genDBConn(c, fromDB) - validator.toDB = conn.NewBaseDB(toDB, func() {}) - validator.toDBConn = d.genDBConn(c, toDB) - fromMock.ExpectQuery("SHOW CREATE TABLE " + fmt.Sprintf("`%s`.`%s`", testCase.schemaName, testCase.tblName)).WillReturnRows( - sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow( - "tbl1", testCase.creatSQL, - ), - ) - fromMock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows( - fromMock.NewRows([]string{"Variable_name", "Value"}).AddRow( - "sql_mode", "", - ), - ) - for i := range testCase.selectSQLs { - rawRetRows := testCase.retRows[i] - rows := sqlmock.NewRows(testCase.colNames) - for j := range rawRetRows { - if len(rawRetRows[j]) == 0 { - break // for delete query - } - rowVals := []driver.Value{} - for k := range rawRetRows[j] { - rowVals = append(rowVals, rawRetRows[j][k]) - } - rows.AddRow(rowVals...) - } - toMock.ExpectQuery(testCase.selectSQLs[i]).WillReturnRows(rows) - } - validator.doValidate() - // wait for all routine finished - time.Sleep(1 * time.Second) - validator.cancel() - validator.wg.Wait() - // failed row - c.Assert(int(validator.failedRowCnt.Load()), Equals, testCase.failRowCnt) - // valid table - fullTableName := testCase.schemaName + "." + testCase.tblName - table, ok := validator.tables[fullTableName] - c.Assert(ok, IsTrue) - c.Assert(len(table.Info.Columns), Equals, len(testCase.colNames)) - if testCase.failRowCnt > 0 { - // validate failed rows - _, ok := validator.failedChangesMap[0][fullTableName] - c.Assert(ok, IsTrue) - allRowsPKs := []string{} - for key := range validator.failedChangesMap[0][fullTableName].rows { - allRowsPKs = append(allRowsPKs, key) - } - sort.Slice(allRowsPKs, func(l, r int) bool { - return allRowsPKs[l] < allRowsPKs[r] - }) - c.Assert(allRowsPKs, DeepEquals, testCase.failRowPKs) - } - c.Assert(len(validator.result.Errors), Equals, 0) + dmlEvents, _, err := generator.GenDMLEvents(replication.WRITE_ROWS_EVENTv2, insertData, 0) + require.NoError(t, err) + updateEvents, _, err := generator.GenDMLEvents(replication.UPDATE_ROWS_EVENTv2, updateData, 0) + require.NoError(t, err) + deleteEvents, _, err := generator.GenDMLEvents(replication.DELETE_ROWS_EVENTv2, deleteData, 0) + require.NoError(t, err) + allEvents := []*replication.BinlogEvent{rotateEvent} + allEvents = append(allEvents, dmlEvents...) + allEvents = append(allEvents, updateEvents...) + allEvents = append(allEvents, deleteEvents...) + mockStreamerProducer := &MockStreamProducer{events: allEvents} + mockStreamer, err := mockStreamerProducer.generateStreamer(binlog.NewLocation("")) + require.NoError(t, err) + + validator := NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + validator.streamerController = &StreamerController{ + streamerProducer: mockStreamerProducer, + streamer: mockStreamer, + closed: false, } + validator.doValidate() + require.Equal(t, int64(1), validator.changeEventCount[rowInsert].Load()) + require.Equal(t, int64(1), validator.changeEventCount[rowUpdated].Load()) + require.Equal(t, int64(1), validator.changeEventCount[rowDeleted].Load()) + ft := filter.Table{Schema: schemaName, Name: tableName2} + require.Contains(t, validator.unsupportedTable, ft.String()) + ft = filter.Table{Schema: schemaName, Name: tableName3} + require.Contains(t, validator.unsupportedTable, ft.String()) +} + +func Test_validator_getRowChangeType(t *testing.T) { + require.Equal(t, rowInsert, getRowChangeType(replication.WRITE_ROWS_EVENTv0)) + require.Equal(t, rowInsert, getRowChangeType(replication.WRITE_ROWS_EVENTv1)) + require.Equal(t, rowInsert, getRowChangeType(replication.WRITE_ROWS_EVENTv2)) + require.Equal(t, rowUpdated, getRowChangeType(replication.UPDATE_ROWS_EVENTv0)) + require.Equal(t, rowUpdated, getRowChangeType(replication.UPDATE_ROWS_EVENTv1)) + require.Equal(t, rowUpdated, getRowChangeType(replication.UPDATE_ROWS_EVENTv2)) + require.Equal(t, rowDeleted, getRowChangeType(replication.DELETE_ROWS_EVENTv0)) + require.Equal(t, rowDeleted, getRowChangeType(replication.DELETE_ROWS_EVENTv1)) + require.Equal(t, rowDeleted, getRowChangeType(replication.DELETE_ROWS_EVENTv2)) } diff --git a/dm/syncer/validate_worker_test.go b/dm/syncer/validate_worker_test.go new file mode 100644 index 00000000000..b9a6cc89572 --- /dev/null +++ b/dm/syncer/validate_worker_test.go @@ -0,0 +1,122 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package syncer + +import ( + "context" + "database/sql/driver" + "testing" + + "github.com/DATA-DOG/go-sqlmock" + "github.com/stretchr/testify/require" +) + +func Test_ValidatorWorker_getTargetRows(t *testing.T) { + type testCase struct { + schemaName string + tblName string + creatSQL string + pkValues [][]string + allCols []string + rowData [][]interface{} + querySQL string + } + testCases := []testCase{ + { + schemaName: "test1", + tblName: "tbl1", + creatSQL: `create table if not exists test1.tbl1( + a int, + b int, + c int, + primary key(a, b) + );`, + pkValues: [][]string{ + {"1", "2"}, {"3", "4"}, {"5", "6"}, + }, + allCols: []string{"a", "b", "c"}, + rowData: [][]interface{}{ + {"1", "2", "3"}, {"3", "4", "5"}, {"5", "6", "7"}, + }, + querySQL: "SELECT .* FROM .*test1.*", + }, + { + schemaName: "test2", + tblName: "tbl2", + creatSQL: `create table if not exists test2.tbl2( + a varchar(10), + other text, + b varbinary(100), + c int, + primary key(a) + );`, + pkValues: [][]string{ + {"a"}, {"b"}, {"c"}, + }, + allCols: []string{"a", "other", "b", "c"}, + rowData: [][]interface{}{ + {"a", "aaa", "\xdd\xcc", "1"}, {"b", "bbb", nil, "2"}, {"c", nil, nil, "3"}, + }, + querySQL: "SELECT .* FROM .*test2.*", + }, + } + db, mock, err := sqlmock.New() + require.NoError(t, err) + for i, tc := range testCases { + var args []driver.Value + for _, arr := range testCases[i].pkValues { + for _, val := range arr { + args = append(args, val) + } + } + dataRows := mock.NewRows(tc.allCols) + for j := range testCases[i].rowData { + var args []driver.Value + for _, val := range tc.rowData[j] { + args = append(args, val) + } + dataRows = dataRows.AddRow(args...) + } + mock.ExpectQuery(tc.querySQL).WithArgs(args...).WillReturnRows(dataRows) + cond := &Cond{ + Table: genValidateTableInfo(t, tc.schemaName, tc.tblName, tc.creatSQL), + ColumnCnt: 2, + PkValues: tc.pkValues, + } + dbConn := genDBConn(t, db, genSubtaskConfig(t)) + + worker := &validateWorker{ + ctx: context.Background(), + conn: dbConn, + } + targetRows, err := worker.getTargetRows(cond) + require.NoError(t, err) + require.Equal(t, 3, len(targetRows)) + for i, pkVs := range tc.pkValues { + key := genRowKey(pkVs) + require.Contains(t, targetRows, key) + data := targetRows[key] + require.Equal(t, len(tc.rowData[i]), len(data)) + for j, val := range tc.rowData[i] { + if val == nil { + require.True(t, data[j].IsNull) + require.Nil(t, data[j].Data) + } else { + require.False(t, data[j].IsNull) + require.Equal(t, val, string(data[j].Data)) + } + } + } + } +} diff --git a/dm/syncer/validator_cond_test.go b/dm/syncer/validator_cond_test.go index 87eee4c8c4b..493814f020b 100644 --- a/dm/syncer/validator_cond_test.go +++ b/dm/syncer/validator_cond_test.go @@ -27,7 +27,7 @@ import ( "github.com/stretchr/testify/require" ) -func getTableDiff(t *testing.T, schemaName, tableName, creatSQL string) *validateTableInfo { +func genValidateTableInfo(t *testing.T, schemaName, tableName, creatSQL string) *validateTableInfo { var ( err error parser2 *parser.Parser @@ -37,16 +37,21 @@ func getTableDiff(t *testing.T, schemaName, tableName, creatSQL string) *validat require.NoError(t, err) tableInfo, err = dbutil.GetTableInfoBySQL(creatSQL, parser2) require.NoError(t, err) - columnMap := make(map[string]*model.ColumnInfo) - for _, col := range tableInfo.Columns { - columnMap[col.Name.O] = col - } var primaryIdx *model.IndexInfo for _, idx := range tableInfo.Indices { if idx.Primary { primaryIdx = idx } } + require.NotNil(t, primaryIdx) + columnMap := make(map[string]*model.ColumnInfo) + for _, col := range tableInfo.Columns { + columnMap[col.Name.O] = col + } + pkIndices := make([]int, len(primaryIdx.Columns)) + for i, col := range primaryIdx.Columns { + pkIndices[i] = columnMap[col.Name.O].Offset + } tableDiff := &validateTableInfo{ Source: &filter.Table{ Schema: schemaName, @@ -54,12 +59,17 @@ func getTableDiff(t *testing.T, schemaName, tableName, creatSQL string) *validat }, Info: tableInfo, PrimaryKey: primaryIdx, + Target: &filter.Table{ + Schema: schemaName, + Name: tableName, + }, + pkIndices: pkIndices, } return tableDiff } -func formatCond(t *testing.T, schemaName, tblName, creatSQL string, pkvs [][]string) *Cond { - tblDiff := getTableDiff(t, schemaName, tblName, creatSQL) +func genValidationCond(t *testing.T, schemaName, tblName, creatSQL string, pkvs [][]string) *Cond { + tblDiff := genValidateTableInfo(t, schemaName, tblName, creatSQL) return &Cond{ Table: tblDiff, PkValues: pkvs, @@ -86,7 +96,7 @@ func TestCondSelectMultiKey(t *testing.T) { key1, key2 := strconv.Itoa(i+1), strconv.Itoa(i+2) pkValues = append(pkValues, []string{key1, key2}) } - cond := formatCond(t, "test_cond", "test1", creatTbl, pkValues) + cond := genValidationCond(t, "test_cond", "test1", creatTbl, pkValues) // format query string rowsQuery := fmt.Sprintf("SELECT COUNT(*) FROM %s WHERE %s;", "`test_cond`.`test1`", cond.GetWhere()) mock.ExpectQuery( @@ -155,7 +165,7 @@ func TestCondGetWhereArgs(t *testing.T) { }, } for i := 0; i < len(cases); i++ { - cond := formatCond(t, cases[i].schemaName, cases[i].tblName, cases[i].creatTbl, cases[i].pks) + cond := genValidationCond(t, cases[i].schemaName, cases[i].tblName, cases[i].creatTbl, cases[i].pks) require.Equal(t, cases[i].where, cond.GetWhere()) rawArgs := cond.GetArgs() for j := 0; j < 3; j++ { From 50c81acbfa94e6b9bda9c6ad268f1703e81aeb87 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Fri, 25 Feb 2022 14:50:41 +0800 Subject: [PATCH 20/36] add ut and fix --- dm/syncer/data_validator.go | 45 ++--- dm/syncer/data_validator_test.go | 58 +++--- dm/syncer/validate_worker.go | 65 ++++-- dm/syncer/validate_worker_test.go | 320 +++++++++++++++++++++++++++++- dm/syncer/validator_cond_test.go | 14 +- 5 files changed, 412 insertions(+), 90 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index d1c2e77bb99..de04503d3bf 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -43,11 +43,8 @@ import ( ) const ( - checkInterval = 5 * time.Second - validationInterval = 10 * time.Second - defaultBatchRowCount = 500 - workerChannelSize = 1000 - defaultWorkerCnt = 4 + checkInterval = 5 * time.Second + validationInterval = 10 * time.Second ) type validateTableInfo struct { @@ -61,28 +58,27 @@ type validateTableInfo struct { type rowChangeType int const ( - rowInvalidChange rowChangeType = iota - rowInsert + rowInsert rowChangeType = iota rowDeleted rowUpdated ) // change of table // binlog changes are clustered into table changes -// the validator validates changes of table-grain at a time +// the validator validates changes of table-grain at a time. type tableChange struct { table *validateTableInfo rows map[string]*rowChange } -// change of a row +// change of a row. type rowChange struct { table *validateTableInfo key string pkValues []string data []interface{} theType rowChangeType - lastMeetTs int64 // the last meet timestamp(in seconds) + lastMeetTS int64 // the last meet timestamp(in seconds) failedCnt int // failed count } @@ -291,10 +287,9 @@ func (v *DataValidator) waitSyncerSynced(currLoc binlog.Location) error { cmp = binlog.CompareLocation(currLoc, syncLoc, v.cfg.EnableGTID) if cmp <= 0 { return nil - } else { - v.waitSyncerTimer.Reset(checkInterval) - fired = false } + v.waitSyncerTimer.Reset(checkInterval) + fired = false } } } @@ -315,7 +310,7 @@ func (v *DataValidator) waitSyncerRunning() error { } } -// doValidate: runs in a separate goroutine +// doValidate: runs in a separate goroutine. func (v *DataValidator) doValidate() { if err := v.waitSyncerRunning(); err != nil { v.errChan <- terror.Annotate(err, "failed to wait syncer running") @@ -365,8 +360,7 @@ func (v *DataValidator) doValidate() { return } - switch ev := e.Event.(type) { - case *replication.RowsEvent: + if ev, ok := e.Event.(*replication.RowsEvent); ok { if err = v.processRowsEvent(e.Header, ev); err != nil { v.L.Warn("failed to process event: ", zap.Reflect("error", err)) v.errChan <- terror.Annotate(err, "failed to process event") @@ -413,16 +407,7 @@ func (v *DataValidator) Stage() pb.Stage { func (v *DataValidator) startValidateWorkers() { v.wg.Add(v.workerCnt) for i := 0; i < v.workerCnt; i++ { - worker := &validateWorker{ - cfg: v.cfg.ValidatorCfg, - ctx: v.ctx, - interval: v.validateInterval, - validator: v, - L: v.L, - conn: v.toDBConns[i], - rowChangeCh: make(chan *rowChange, workerChannelSize), - pendingChangesMap: make(map[string]*tableChange), - } + worker := newValidateWorker(v, i) v.workers[i] = worker go func() { v.wg.Done() @@ -535,7 +520,7 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re pkValues: pkValue, data: row, theType: rowDeleted, - lastMeetTs: int64(header.Timestamp), + lastMeetTS: int64(header.Timestamp), }) afterRowChangeType = rowInsert } @@ -545,7 +530,7 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re pkValues: afterPkValue, data: afterRow, theType: afterRowChangeType, - lastMeetTs: int64(header.Timestamp), + lastMeetTS: int64(header.Timestamp), }) } else { v.dispatchRowChange(key, &rowChange{ @@ -554,14 +539,14 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re pkValues: pkValue, data: row, theType: changeType, - lastMeetTs: int64(header.Timestamp), + lastMeetTS: int64(header.Timestamp), }) } } return nil } -// getRowChangeType should be called only when the event type is RowsEvent +// getRowChangeType should be called only when the event type is RowsEvent. func getRowChangeType(t replication.EventType) rowChangeType { switch t { case replication.WRITE_ROWS_EVENTv0, replication.WRITE_ROWS_EVENTv1, replication.WRITE_ROWS_EVENTv2: diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index 1b125baf134..82d9b7a2bef 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -21,7 +21,6 @@ import ( "github.com/DATA-DOG/go-sqlmock" "github.com/go-mysql-org/go-mysql/mysql" - gmysql "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/filter" @@ -41,6 +40,7 @@ import ( ) func genEventGenerator(t *testing.T) *event.Generator { + t.Helper() previousGTIDSetStr := "3ccc475b-2343-11e7-be21-6c0b84d59f30:1-14" previousGTIDSet, err := gtid.ParserGTID(mysql.MySQLFlavor, previousGTIDSetStr) require.NoError(t, err) @@ -55,6 +55,7 @@ func genEventGenerator(t *testing.T) *event.Generator { } func genSubtaskConfig(t *testing.T) *config.SubTaskConfig { + t.Helper() loaderCfg := config.LoaderConfig{ Dir: t.TempDir(), } @@ -87,6 +88,7 @@ func genSubtaskConfig(t *testing.T) *config.SubTaskConfig { } func genDBConn(t *testing.T, db *sql.DB, cfg *config.SubTaskConfig) *dbconn.DBConn { + t.Helper() baseDB := conn.NewBaseDB(db, func() {}) baseConn, err := baseDB.GetBaseConn(context.Background()) require.NoError(t, err) @@ -132,6 +134,7 @@ func Test_validator_StartStop(t *testing.T) { // normal start & stop validator = NewContinuousDataValidator(cfg, syncerObj) validator.Start(pb.Stage_Running) + defer validator.Stop() // in case assert failed before Stop require.Equal(t, pb.Stage_Running, validator.Stage()) require.True(t, validator.Started()) validator.Stop() @@ -153,6 +156,7 @@ func Test_validator_fillResult(t *testing.T) { validator := NewContinuousDataValidator(cfg, syncerObj) validator.Start(pb.Stage_Running) + defer validator.Stop() // in case assert failed before Stop validator.fillResult(errors.New("test error"), false) require.Len(t, validator.result.Errors, 1) validator.fillResult(errors.New("test error"), true) @@ -173,6 +177,7 @@ func Test_validator_errorProcessRoutine(t *testing.T) { validator := NewContinuousDataValidator(cfg, syncerObj) validator.Start(pb.Stage_Running) + defer validator.Stop() require.Equal(t, pb.Stage_Running, validator.Stage()) validator.errChan <- errors.New("test error") require.True(t, utils.WaitSomething(20, 100*time.Millisecond, func() bool { @@ -192,9 +197,8 @@ func (c *mockedCheckPointForValidator) FlushedGlobalPoint() binlog.Location { c.cnt++ if c.cnt <= 2 { return c.currLoc - } else { - return c.nextLoc } + return c.nextLoc } func Test_validator_waitSyncerSynced(t *testing.T) { @@ -212,7 +216,7 @@ func Test_validator_waitSyncerSynced(t *testing.T) { require.NoError(t, validator.waitSyncerSynced(currLoc)) // cancelled - currLoc.Position = gmysql.Position{ + currLoc.Position = mysql.Position{ Name: "mysql-bin.000001", Pos: 100, } @@ -221,7 +225,7 @@ func Test_validator_waitSyncerSynced(t *testing.T) { validator.cancel() require.ErrorIs(t, validator.waitSyncerSynced(currLoc), context.Canceled) - currLoc.Position = gmysql.Position{ + currLoc.Position = mysql.Position{ Name: "mysql-bin.000001", Pos: 100, } @@ -265,14 +269,14 @@ func Test_validator_waitSyncerRunning(t *testing.T) { func Test_validator_doValidate(t *testing.T) { var ( - schemaName = "test" - tableName = "tbl" - tableName2 = "tbl2" - tableName3 = "tbl3" - tableName4 = "tbl4" - createTableSql = "CREATE TABLE `" + tableName + "`(id int primary key, v varchar(100))" - createTableSql2 = "CREATE TABLE `" + tableName2 + "`(id int primary key)" - createTableSql3 = "CREATE TABLE `" + tableName3 + "`(id int, v varchar(100))" + schemaName = "test" + tableName = "tbl" + tableName2 = "tbl2" + tableName3 = "tbl3" + tableName4 = "tbl4" + createTableSQL = "CREATE TABLE `" + tableName + "`(id int primary key, v varchar(100))" + createTableSQL2 = "CREATE TABLE `" + tableName2 + "`(id int primary key)" + createTableSQL3 = "CREATE TABLE `" + tableName3 + "`(id int, v varchar(100))" ) cfg := genSubtaskConfig(t) _, _, err := conn.InitMockDBFull() @@ -286,7 +290,7 @@ func Test_validator_doValidate(t *testing.T) { syncerObj.tableRouter, err = router.NewRouter(cfg.CaseSensitive, []*router.TableRule{}) require.NoError(t, err) currLoc := binlog.NewLocation(cfg.Flavor) - currLoc.Position = gmysql.Position{ + currLoc.Position = mysql.Position{ Name: "mysql-bin.000001", Pos: 3000, } @@ -297,11 +301,6 @@ func Test_validator_doValidate(t *testing.T) { } db, mock, err := sqlmock.New() require.NoError(t, err) - //mock.ExpectQuery("SHOW CREATE TABLE.*").WillReturnRows( - // sqlmock.NewRows([]string{"Table", "Create Table"}). - // AddRow(tableName, createTableSql). - // AddRow(tableName2, createTableSql2), - //) mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows( mock.NewRows([]string{"Variable_name", "Value"}).AddRow("sql_mode", ""), ) @@ -309,10 +308,11 @@ func Test_validator_doValidate(t *testing.T) { require.NoError(t, err) syncerObj.downstreamTrackConn = &dbconn.DBConn{Cfg: cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} syncerObj.schemaTracker, err = schema.NewTracker(context.Background(), cfg.Name, defaultTestSessionCfg, syncerObj.downstreamTrackConn) + require.NoError(t, err) require.NoError(t, syncerObj.schemaTracker.CreateSchemaIfNotExists(schemaName)) - require.NoError(t, syncerObj.schemaTracker.Exec(context.Background(), schemaName, createTableSql)) - require.NoError(t, syncerObj.schemaTracker.Exec(context.Background(), schemaName, createTableSql2)) - require.NoError(t, syncerObj.schemaTracker.Exec(context.Background(), schemaName, createTableSql3)) + require.NoError(t, syncerObj.schemaTracker.Exec(context.Background(), schemaName, createTableSQL)) + require.NoError(t, syncerObj.schemaTracker.Exec(context.Background(), schemaName, createTableSQL2)) + require.NoError(t, syncerObj.schemaTracker.Exec(context.Background(), schemaName, createTableSQL3)) generator := genEventGenerator(t) rotateEvent, _, err := generator.Rotate("mysql-bin.000001", 0) @@ -322,7 +322,7 @@ func Test_validator_doValidate(t *testing.T) { TableID: 11, Schema: schemaName, Table: tableName, - ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + ColumnType: []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, Rows: [][]interface{}{ {int32(1), "a"}, {int32(2), "b"}, @@ -334,7 +334,7 @@ func Test_validator_doValidate(t *testing.T) { TableID: 12, Schema: schemaName, Table: tableName2, - ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + ColumnType: []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, Rows: [][]interface{}{ {int32(1), "a"}, {int32(2), "b"}, @@ -346,7 +346,7 @@ func Test_validator_doValidate(t *testing.T) { TableID: 13, Schema: schemaName, Table: tableName3, - ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + ColumnType: []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, Rows: [][]interface{}{ {int32(1), "a"}, {int32(2), "b"}, @@ -358,7 +358,7 @@ func Test_validator_doValidate(t *testing.T) { TableID: 13, Schema: schemaName, Table: tableName3, - ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + ColumnType: []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, Rows: [][]interface{}{ {int32(4), "a"}, }, @@ -369,7 +369,7 @@ func Test_validator_doValidate(t *testing.T) { TableID: 11, Schema: schemaName, Table: tableName, - ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + ColumnType: []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, Rows: [][]interface{}{ // update non-primary column {int32(3), "c"}, @@ -385,7 +385,7 @@ func Test_validator_doValidate(t *testing.T) { TableID: 11, Schema: schemaName, Table: tableName, - ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + ColumnType: []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, Rows: [][]interface{}{ {int32(3), "c"}, }, @@ -395,7 +395,7 @@ func Test_validator_doValidate(t *testing.T) { TableID: 14, Schema: schemaName, Table: tableName4, - ColumnType: []byte{gmysql.MYSQL_TYPE_LONG, gmysql.MYSQL_TYPE_STRING}, + ColumnType: []byte{mysql.MYSQL_TYPE_LONG, mysql.MYSQL_TYPE_STRING}, Rows: [][]interface{}{ {int32(4), "c"}, }, diff --git a/dm/syncer/validate_worker.go b/dm/syncer/validate_worker.go index 611ecacde5c..56fc0454ed8 100644 --- a/dm/syncer/validate_worker.go +++ b/dm/syncer/validate_worker.go @@ -20,6 +20,7 @@ import ( "math" "strconv" "strings" + "sync" "time" "github.com/pingcap/errors" @@ -27,6 +28,7 @@ import ( "github.com/pingcap/tidb/parser/model" tidbmysql "github.com/pingcap/tidb/parser/mysql" "github.com/shopspring/decimal" + "go.uber.org/atomic" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/dm/config" @@ -36,6 +38,10 @@ import ( "github.com/pingcap/tiflow/dm/syncer/dbconn" ) +const ( + workerChannelSize = 1000 +) + type validateWorker struct { cfg config.ValidatorConfig ctx context.Context @@ -45,7 +51,25 @@ type validateWorker struct { conn *dbconn.DBConn rowChangeCh chan *rowChange pendingChangesMap map[string]*tableChange - rowCount int64 + pendingRowCount atomic.Int64 + sync.Mutex + + // used for test + receivedRowCount atomic.Int64 // number of rowChange received from channel + validationCount atomic.Int64 // number of successful validation +} + +func newValidateWorker(v *DataValidator, id int) *validateWorker { + return &validateWorker{ + cfg: v.cfg.ValidatorCfg, + ctx: v.ctx, + interval: v.validateInterval, + validator: v, + L: v.L, + conn: v.toDBConns[id], + rowChangeCh: make(chan *rowChange, workerChannelSize), + pendingChangesMap: make(map[string]*tableChange), + } } func (vw *validateWorker) run() { @@ -54,6 +78,7 @@ func (vw *validateWorker) run() { case change := <-vw.rowChangeCh: // todo: limit number of pending rows vw.updateRowChange(change) + vw.receivedRowCount.Inc() case <-vw.ctx.Done(): return case <-time.After(vw.interval): @@ -62,6 +87,7 @@ func (vw *validateWorker) run() { vw.validator.errChan <- terror.Annotate(err, "failed to validate table change") return } + vw.validationCount.Inc() } } } @@ -81,16 +107,17 @@ func (vw *validateWorker) updateRowChange(row *rowChange) { if val, ok := change.rows[row.key]; ok { val.data = row.data val.theType = row.theType - val.lastMeetTs = row.lastMeetTs + val.lastMeetTS = row.lastMeetTS val.failedCnt = 0 // clear failed count } else { change.rows[row.key] = row - vw.rowCount++ + vw.pendingRowCount.Inc() } } func (vw *validateWorker) validateTableChange() error { failedChanges := make(map[string]*tableChange) + var failedRowCnt int64 for k, val := range vw.pendingChangesMap { var insertUpdateChanges, deleteChanges []*rowChange for _, r := range val.rows { @@ -100,7 +127,7 @@ func (vw *validateWorker) validateTableChange() error { insertUpdateChanges = append(insertUpdateChanges, r) } } - rows := make(map[string]*rowChange, 0) + rows := make(map[string]*rowChange) if len(insertUpdateChanges) > 0 { // todo: limit number of validated rows failedRows, err := vw.validateRowChanges(val.table, insertUpdateChanges, false) @@ -128,8 +155,10 @@ func (vw *validateWorker) validateTableChange() error { table: val.table, rows: rows, } + failedRowCnt += int64(len(rows)) } } + vw.pendingRowCount.Store(failedRowCnt) vw.pendingChangesMap = failedChanges return nil } @@ -205,34 +234,30 @@ func (vw *validateWorker) validateInsertAndUpdateRows(rows []*rowChange, cond *C } func (vw *validateWorker) compareData(sourceData, targetData []*dbutil.ColumnData, columns []*model.ColumnInfo) (bool, error) { - equal := true for i, column := range columns { data1, data2 := sourceData[i], targetData[i] + if data1.IsNull != data2.IsNull { + return false, nil + } str1, str2 := string(data1.Data), string(data2.Data) - if column.FieldType.Tp == tidbmysql.TypeFloat || column.FieldType.Tp == tidbmysql.TypeDouble { - if data1.IsNull == data2.IsNull && data1.IsNull { - continue - } - + if str1 == str2 { + continue + } else if column.FieldType.Tp == tidbmysql.TypeFloat || column.FieldType.Tp == tidbmysql.TypeDouble { + // source and target data have different precision? num1, err1 := strconv.ParseFloat(str1, 64) num2, err2 := strconv.ParseFloat(str2, 64) if err1 != nil || err2 != nil { + // should not happen return false, errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", str1, str2, err1, err2) } if math.Abs(num1-num2) <= 1e-6 { continue } - } else { - if (str1 == str2) && (data1.IsNull == data2.IsNull) { - continue - } } - - equal = false - break + return false, nil } - return equal, nil + return true, nil } func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.ColumnData, error) { @@ -256,7 +281,7 @@ func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.Column } defer rows.Close() - result := make(map[string][]*dbutil.ColumnData, 0) + result := make(map[string][]*dbutil.ColumnData) for rows.Next() { rowData, err := scanRow(rows) if err != nil { @@ -270,7 +295,7 @@ func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.Column pk := genRowKey(pkVals) result[pk] = rowData } - return result, nil + return result, rows.Err() } func scanRow(rows *sql.Rows) ([]*dbutil.ColumnData, error) { diff --git a/dm/syncer/validate_worker_test.go b/dm/syncer/validate_worker_test.go index b9a6cc89572..7e263f5515c 100644 --- a/dm/syncer/validate_worker_test.go +++ b/dm/syncer/validate_worker_test.go @@ -16,12 +16,270 @@ package syncer import ( "context" "database/sql/driver" + "sync" "testing" + "time" "github.com/DATA-DOG/go-sqlmock" + "github.com/pingcap/errors" + "github.com/pingcap/tidb-tools/pkg/dbutil" + "github.com/pingcap/tidb-tools/pkg/filter" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/types" + "github.com/shopspring/decimal" "github.com/stretchr/testify/require" + + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/utils" ) +// split into 3 cases, since it may be unstable when put together. +func Test_ValidatorWorker_run_insert_update(t *testing.T) { + tbl1 := filter.Table{Schema: "test", Name: "tbl1"} + tableInfo1 := genValidateTableInfo(t, tbl1.Schema, tbl1.Name, + "create table tbl1(a int primary key, b varchar(100))") + + cfg := genSubtaskConfig(t) + _, mock, err := conn.InitMockDBFull() + require.NoError(t, err) + defer func() { + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + }() + syncerObj := NewSyncer(cfg, nil, nil) + validator := NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + defer validator.cancel() + + // insert & update same table, one success, one fail + worker := newValidateWorker(validator, 0) + var wg sync.WaitGroup + wg.Add(1) + go func() { + wg.Done() + worker.run() + }() + worker.rowChangeCh <- &rowChange{ + table: tableInfo1, + key: "1", + pkValues: []string{"1"}, + data: []interface{}{1, "a"}, + theType: rowInsert, + lastMeetTS: time.Now().Unix(), + } + worker.rowChangeCh <- &rowChange{ + table: tableInfo1, + key: "1", + pkValues: []string{"1"}, + data: []interface{}{1, "b"}, + theType: rowUpdated, + lastMeetTS: time.Now().Unix(), + } + worker.rowChangeCh <- &rowChange{ + table: tableInfo1, + key: "2", + pkValues: []string{"2"}, + data: []interface{}{2, "2b"}, + theType: rowInsert, + lastMeetTS: time.Now().Unix(), + } + mock.ExpectQuery("SELECT .* FROM .*tbl1.* WHERE .*").WillReturnRows( + sqlmock.NewRows([]string{"a", "b"}).AddRow(2, "incorrect data")) + require.True(t, utils.WaitSomething(30, time.Second, func() bool { + return worker.receivedRowCount.Load() == 3 + })) + require.True(t, utils.WaitSomething(30, time.Second, func() bool { + return worker.validationCount.Load() > 0 + })) + validator.cancel() + wg.Wait() + require.Equal(t, int64(3), worker.receivedRowCount.Load()) + require.Equal(t, int64(2), worker.pendingRowCount.Load()) + require.Len(t, worker.pendingChangesMap, 1) + require.Contains(t, worker.pendingChangesMap, tbl1.String()) + require.Len(t, worker.pendingChangesMap[tbl1.String()].rows, 2) + require.Contains(t, worker.pendingChangesMap[tbl1.String()].rows, "1") + require.Equal(t, rowUpdated, worker.pendingChangesMap[tbl1.String()].rows["1"].theType) + require.Equal(t, int(worker.validationCount.Load()), worker.pendingChangesMap[tbl1.String()].rows["1"].failedCnt) + require.Contains(t, worker.pendingChangesMap[tbl1.String()].rows, "2") + require.Equal(t, rowInsert, worker.pendingChangesMap[tbl1.String()].rows["2"].theType) + require.Equal(t, int(worker.validationCount.Load()), worker.pendingChangesMap[tbl1.String()].rows["2"].failedCnt) +} + +func Test_ValidatorWorker_run_all_validated(t *testing.T) { + tbl1 := filter.Table{Schema: "test", Name: "tbl1"} + tableInfo1 := genValidateTableInfo(t, tbl1.Schema, tbl1.Name, + "create table tbl1(a int primary key, b varchar(100))") + + cfg := genSubtaskConfig(t) + _, mock, err := conn.InitMockDBFull() + require.NoError(t, err) + defer func() { + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + }() + syncerObj := NewSyncer(cfg, nil, nil) + validator := NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + defer validator.cancel() + + // insert & update same table, one success, one fail + worker := newValidateWorker(validator, 0) + var wg sync.WaitGroup + wg.Add(1) + go func() { + wg.Done() + worker.run() + }() + worker.rowChangeCh <- &rowChange{ + table: tableInfo1, + key: "1", + pkValues: []string{"1"}, + data: []interface{}{1, "a"}, + theType: rowInsert, + lastMeetTS: time.Now().Unix(), + } + worker.rowChangeCh <- &rowChange{ + table: tableInfo1, + key: "1", + pkValues: []string{"1"}, + data: []interface{}{1, "b"}, + theType: rowUpdated, + lastMeetTS: time.Now().Unix(), + } + worker.rowChangeCh <- &rowChange{ + table: tableInfo1, + key: "2", + pkValues: []string{"2"}, + data: []interface{}{2, "2b"}, + theType: rowInsert, + lastMeetTS: time.Now().Unix(), + } + mock.ExpectQuery("SELECT .* FROM .*tbl1.* WHERE .*").WillReturnRows( + sqlmock.NewRows([]string{"a", "b"}).AddRow(1, "b").AddRow(2, "2b")) + require.True(t, utils.WaitSomething(30, time.Second, func() bool { + return worker.receivedRowCount.Load() == 3 + })) + require.True(t, utils.WaitSomething(30, time.Second, func() bool { + return worker.validationCount.Load() > 0 + })) + validator.cancel() + wg.Wait() + require.Equal(t, int64(3), worker.receivedRowCount.Load()) + require.Equal(t, int64(0), worker.pendingRowCount.Load()) + require.Len(t, worker.pendingChangesMap, 0) +} + +func Test_ValidatorWorker_run_delete(t *testing.T) { + tbl2 := filter.Table{Schema: "test", Name: "tbl2"} + tbl3 := filter.Table{Schema: "test", Name: "tbl3"} + tableInfo2 := genValidateTableInfo(t, tbl2.Schema, tbl2.Name, + "create table tbl2(a varchar(100) primary key, b varchar(100))") + tableInfo3 := genValidateTableInfo(t, tbl3.Schema, tbl3.Name, + "create table tbl3(a varchar(100) primary key, b varchar(100))") + + cfg := genSubtaskConfig(t) + _, mock, err := conn.InitMockDBFull() + require.NoError(t, err) + defer func() { + conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} + }() + syncerObj := NewSyncer(cfg, nil, nil) + validator := NewContinuousDataValidator(cfg, syncerObj) + validator.Start(pb.Stage_Stopped) + defer validator.cancel() + + worker := newValidateWorker(validator, 0) + var wg sync.WaitGroup + wg.Add(1) + go func() { + wg.Done() + worker.run() + }() + worker.rowChangeCh <- &rowChange{ + table: tableInfo2, + key: "a", + pkValues: []string{"a"}, + data: []interface{}{"a", "b"}, + theType: rowDeleted, + lastMeetTS: time.Now().Unix(), + } + worker.rowChangeCh <- &rowChange{ + table: tableInfo3, + key: "aa", + pkValues: []string{"aa"}, + data: []interface{}{"aa", "b"}, + theType: rowDeleted, + lastMeetTS: time.Now().Unix(), + } + mock.ExpectQuery("SELECT .* FROM .*tbl2.* WHERE .*").WillReturnRows( + sqlmock.NewRows([]string{"a", "b"})) + mock.ExpectQuery("SELECT .* FROM .*tbl3.* WHERE .*").WillReturnRows( + sqlmock.NewRows([]string{"a", "b"}).AddRow("aa", "b")) + + // wait all events received by worker + require.True(t, utils.WaitSomething(30, time.Second, func() bool { + return worker.receivedRowCount.Load() == 2 + })) + currCnt := worker.validationCount.Load() + require.True(t, utils.WaitSomething(30, time.Second, func() bool { + return worker.validationCount.Load() > currCnt + })) + validator.cancel() + wg.Wait() + + require.Equal(t, int64(1), worker.pendingRowCount.Load()) + require.Len(t, worker.pendingChangesMap, 1) + require.Contains(t, worker.pendingChangesMap, tbl3.String()) + require.Len(t, worker.pendingChangesMap[tbl3.String()].rows, 1) + require.Contains(t, worker.pendingChangesMap[tbl3.String()].rows, "aa") + require.Equal(t, rowDeleted, worker.pendingChangesMap[tbl3.String()].rows["aa"].theType) +} + +func Test_ValidatorWorker_compareData(t *testing.T) { + worker := validateWorker{} + eq, err := worker.compareData([]*dbutil.ColumnData{{Data: []byte("1")}}, + []*dbutil.ColumnData{{IsNull: true}}, + []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeLong}}}) + require.NoError(t, err) + require.False(t, eq) + eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("1.1")}}, + []*dbutil.ColumnData{{Data: []byte("1.x")}}, + []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeFloat}}}) + require.Error(t, err) + require.False(t, eq) + eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("1.1")}}, + []*dbutil.ColumnData{{Data: []byte("1.1000011")}}, + []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeFloat}}}) + require.NoError(t, err) + require.False(t, eq) + eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("1.1")}}, + []*dbutil.ColumnData{{Data: []byte("1.1000001")}}, + []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeFloat}}}) + require.NoError(t, err) + require.True(t, eq) + eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("1.1")}}, + []*dbutil.ColumnData{{Data: []byte("1.1000001")}}, + []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeDouble}}}) + require.NoError(t, err) + require.True(t, eq) + eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("1")}}, + []*dbutil.ColumnData{{Data: []byte("1")}}, + []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeLong}}}) + require.NoError(t, err) + require.True(t, eq) + eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("aaa")}}, + []*dbutil.ColumnData{{Data: []byte("aaa")}}, + []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeVarchar}}}) + require.NoError(t, err) + require.True(t, eq) + eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("\x01\x02")}}, + []*dbutil.ColumnData{{Data: []byte("\x01\x02")}}, + []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeVarString}}}) + require.NoError(t, err) + require.True(t, eq) +} + func Test_ValidatorWorker_getTargetRows(t *testing.T) { type testCase struct { schemaName string @@ -82,11 +340,11 @@ func Test_ValidatorWorker_getTargetRows(t *testing.T) { } dataRows := mock.NewRows(tc.allCols) for j := range testCases[i].rowData { - var args []driver.Value + var rowData []driver.Value for _, val := range tc.rowData[j] { - args = append(args, val) + rowData = append(rowData, val) } - dataRows = dataRows.AddRow(args...) + dataRows = dataRows.AddRow(rowData...) } mock.ExpectQuery(tc.querySQL).WithArgs(args...).WillReturnRows(dataRows) cond := &Cond{ @@ -100,8 +358,8 @@ func Test_ValidatorWorker_getTargetRows(t *testing.T) { ctx: context.Background(), conn: dbConn, } - targetRows, err := worker.getTargetRows(cond) - require.NoError(t, err) + targetRows, err2 := worker.getTargetRows(cond) + require.NoError(t, err2) require.Equal(t, 3, len(targetRows)) for i, pkVs := range tc.pkValues { key := genRowKey(pkVs) @@ -119,4 +377,56 @@ func Test_ValidatorWorker_getTargetRows(t *testing.T) { } } } + + cond := &Cond{ + Table: genValidateTableInfo(t, "test", "tbl", "create table tbl(a int primary key)"), + ColumnCnt: 1, + PkValues: [][]string{{"1"}}, + } + worker := &validateWorker{ + ctx: context.Background(), + conn: genDBConn(t, db, genSubtaskConfig(t)), + } + + // query error + mock.ExpectQuery("SELECT .* FROM .*").WithArgs(sqlmock.AnyArg()).WillReturnError(errors.New("query")) + _, err = worker.getTargetRows(cond) + require.EqualError(t, errors.Cause(err), "query") +} + +func Test_ValidatorWorker_getSourceRowsForCompare(t *testing.T) { + rows := getSourceRowsForCompare([]*rowChange{ + { + key: "a", + data: []interface{}{ + nil, 1, + }, + }, + { + key: "b", + data: []interface{}{ + 1, 2, + }, + }, + }) + require.Len(t, rows, 2) + require.Len(t, rows["a"], 2) + require.Len(t, rows["b"], 2) + require.True(t, rows["a"][0].IsNull) + require.Equal(t, []byte("1"), rows["a"][1].Data) + require.Equal(t, []byte("1"), rows["b"][0].Data) + require.Equal(t, []byte("2"), rows["b"][1].Data) +} + +func Test_ValidatorWorker_genColData(t *testing.T) { + res := genColData(1) + require.Equal(t, "1", string(res)) + res = genColData(1.2) + require.Equal(t, "1.2", string(res)) + res = genColData("abc") + require.Equal(t, "abc", string(res)) + res = genColData([]byte{'\x01', '\x02', '\x03'}) + require.Equal(t, "\x01\x02\x03", string(res)) + res = genColData(decimal.NewFromInt(222123123)) + require.Equal(t, "222123123", string(res)) } diff --git a/dm/syncer/validator_cond_test.go b/dm/syncer/validator_cond_test.go index 493814f020b..66b2f848f5f 100644 --- a/dm/syncer/validator_cond_test.go +++ b/dm/syncer/validator_cond_test.go @@ -28,6 +28,7 @@ import ( ) func genValidateTableInfo(t *testing.T, schemaName, tableName, creatSQL string) *validateTableInfo { + t.Helper() var ( err error parser2 *parser.Parser @@ -60,15 +61,16 @@ func genValidateTableInfo(t *testing.T, schemaName, tableName, creatSQL string) Info: tableInfo, PrimaryKey: primaryIdx, Target: &filter.Table{ - Schema: schemaName, - Name: tableName, - }, + Schema: schemaName, + Name: tableName, + }, pkIndices: pkIndices, } return tableDiff } func genValidationCond(t *testing.T, schemaName, tblName, creatSQL string, pkvs [][]string) *Cond { + t.Helper() tblDiff := genValidateTableInfo(t, schemaName, tblName, creatSQL) return &Cond{ Table: tblDiff, @@ -77,9 +79,7 @@ func genValidationCond(t *testing.T, schemaName, tblName, creatSQL string, pkvs } func TestCondSelectMultiKey(t *testing.T) { - var ( - res *sql.Rows - ) + var res *sql.Rows db, mock, err := sqlmock.New() require.NoError(t, err) defer db.Close() @@ -107,12 +107,14 @@ func TestCondSelectMultiKey(t *testing.T) { require.NoError(t, err) res, err = db.Query(rowsQuery, cond.GetArgs()...) require.NoError(t, err) + defer res.Close() var cnt int if res.Next() { err = res.Scan(&cnt) } require.NoError(t, err) require.Equal(t, 3, cnt) + require.NoError(t, res.Err()) } func TestCondGetWhereArgs(t *testing.T) { From e8ca5a101897e680eaf7d8516b042d3642317f85 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Fri, 25 Feb 2022 15:25:23 +0800 Subject: [PATCH 21/36] ut and fix --- dm/syncer/data_validator.go | 14 +++----------- dm/syncer/validate_worker.go | 1 + dm/syncer/validator_cond.go | 16 ++++++++-------- dm/syncer/validator_cond_test.go | 4 ++-- 4 files changed, 14 insertions(+), 21 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index de04503d3bf..b83ecc8c44b 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -15,8 +15,6 @@ package syncer import ( "context" - "fmt" - "hash/fnv" "strings" "sync" "time" @@ -416,14 +414,8 @@ func (v *DataValidator) startValidateWorkers() { } } -func hashTablePk(s string) uint32 { - h := fnv.New32a() - h.Write([]byte(s)) - return h.Sum32() -} - func (v *DataValidator) dispatchRowChange(key string, row *rowChange) { - hashVal := int(hashTablePk(key)) % v.workerCnt + hashVal := int(utils.GenHashKey(key)) % v.workerCnt v.workers[hashVal].rowChangeCh <- row } @@ -500,7 +492,7 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re row := ev.Rows[i] pkValue := make([]string, len(pk.Columns)) for _, idx := range pkIndices { - pkValue[idx] = fmt.Sprintf("%v", row[idx]) + pkValue[idx] = string(genColData(row[idx])) } key := genRowKey(pkValue) @@ -509,7 +501,7 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re afterRow := ev.Rows[i+1] afterPkValue := make([]string, len(pk.Columns)) for _, idx := range pkIndices { - afterPkValue[idx] = fmt.Sprintf("%v", afterRow[idx]) + afterPkValue[idx] = string(genColData(afterRow[idx])) } afterKey := genRowKey(afterPkValue) if afterKey != key { diff --git a/dm/syncer/validate_worker.go b/dm/syncer/validate_worker.go index 56fc0454ed8..13a07746da8 100644 --- a/dm/syncer/validate_worker.go +++ b/dm/syncer/validate_worker.go @@ -84,6 +84,7 @@ func (vw *validateWorker) run() { case <-time.After(vw.interval): err := vw.validateTableChange() if err != nil { + // todo: better error handling vw.validator.errChan <- terror.Annotate(err, "failed to validate table change") return } diff --git a/dm/syncer/validator_cond.go b/dm/syncer/validator_cond.go index ffe9d5250d0..247c8a2e0fb 100644 --- a/dm/syncer/validator_cond.go +++ b/dm/syncer/validator_cond.go @@ -37,16 +37,16 @@ func (c *Cond) GetWhere() string { var b strings.Builder pk := c.Table.PrimaryKey isOneKey := len(pk.Columns) == 1 - if !isOneKey { + if isOneKey { + b.WriteString(pk.Columns[0].Name.O) + } else { b.WriteString("(") - } - for i := 0; i < len(pk.Columns); i++ { - if i != 0 { - b.WriteString(",") + for i := 0; i < len(pk.Columns); i++ { + if i != 0 { + b.WriteString(",") + } + b.WriteString(pk.Columns[i].Name.O) } - b.WriteString(pk.Columns[i].Name.O) - } - if !isOneKey { b.WriteString(")") } b.WriteString(" in (") diff --git a/dm/syncer/validator_cond_test.go b/dm/syncer/validator_cond_test.go index 66b2f848f5f..7167d3f2957 100644 --- a/dm/syncer/validator_cond_test.go +++ b/dm/syncer/validator_cond_test.go @@ -78,7 +78,7 @@ func genValidationCond(t *testing.T, schemaName, tblName, creatSQL string, pkvs } } -func TestCondSelectMultiKey(t *testing.T) { +func Test_validatorCond_SelectMultiKey(t *testing.T) { var res *sql.Rows db, mock, err := sqlmock.New() require.NoError(t, err) @@ -117,7 +117,7 @@ func TestCondSelectMultiKey(t *testing.T) { require.NoError(t, res.Err()) } -func TestCondGetWhereArgs(t *testing.T) { +func Test_validatorCond_GetWhereArgs(t *testing.T) { db, _, err := sqlmock.New() require.NoError(t, err) defer db.Close() From 8eb156d40225083e42ddfa4e8b3561451151f159 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Fri, 25 Feb 2022 16:27:54 +0800 Subject: [PATCH 22/36] remove useless field --- dm/syncer/data_validator.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index b83ecc8c44b..3796e24f9f1 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -112,7 +112,6 @@ type DataValidator struct { result pb.ProcessResult validateInterval time.Duration workers []*validateWorker - rowsEventChan chan *replication.BinlogEvent // unbuffered is enough changeEventCount []atomic.Int64 workerCnt int @@ -133,7 +132,6 @@ func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *D v.workerCnt = cfg.ValidatorCfg.WorkerCount v.workers = make([]*validateWorker, v.workerCnt) - v.rowsEventChan = make(chan *replication.BinlogEvent) v.changeEventCount = make([]atomic.Int64, 4) v.validateInterval = validationInterval From 70bea8da3d238031dbabe4aefab1324bdbde3714 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Fri, 25 Feb 2022 16:31:35 +0800 Subject: [PATCH 23/36] move genColData --- dm/syncer/data_validator.go | 15 +++++++++++++++ dm/syncer/data_validator_test.go | 14 ++++++++++++++ dm/syncer/validate_worker.go | 14 -------------- dm/syncer/validate_worker_test.go | 14 -------------- 4 files changed, 29 insertions(+), 28 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 3796e24f9f1..2d4a33bb30a 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -15,6 +15,7 @@ package syncer import ( "context" + "fmt" "strings" "sync" "time" @@ -23,6 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser/model" + "github.com/shopspring/decimal" "go.uber.org/atomic" "go.uber.org/zap" @@ -552,3 +554,16 @@ func getRowChangeType(t replication.EventType) rowChangeType { func genRowKey(pkValues []string) string { return strings.Join(pkValues, "-") } + +func genColData(v interface{}) []byte { + switch dv := v.(type) { + case []byte: + return dv + case string: + return []byte(dv) + case decimal.Decimal: + return []byte(dv.String()) + } + s := fmt.Sprintf("%v", v) + return []byte(s) +} diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index 82d9b7a2bef..6f2aeacb6a1 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -24,6 +24,7 @@ import ( "github.com/go-mysql-org/go-mysql/replication" "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/filter" + "github.com/shopspring/decimal" "github.com/stretchr/testify/require" "github.com/pingcap/tiflow/dm/dm/config" @@ -443,3 +444,16 @@ func Test_validator_getRowChangeType(t *testing.T) { require.Equal(t, rowDeleted, getRowChangeType(replication.DELETE_ROWS_EVENTv1)) require.Equal(t, rowDeleted, getRowChangeType(replication.DELETE_ROWS_EVENTv2)) } + +func Test_validator_genColData(t *testing.T) { + res := genColData(1) + require.Equal(t, "1", string(res)) + res = genColData(1.2) + require.Equal(t, "1.2", string(res)) + res = genColData("abc") + require.Equal(t, "abc", string(res)) + res = genColData([]byte{'\x01', '\x02', '\x03'}) + require.Equal(t, "\x01\x02\x03", string(res)) + res = genColData(decimal.NewFromInt(222123123)) + require.Equal(t, "222123123", string(res)) +} diff --git a/dm/syncer/validate_worker.go b/dm/syncer/validate_worker.go index 13a07746da8..bc8c313c6b8 100644 --- a/dm/syncer/validate_worker.go +++ b/dm/syncer/validate_worker.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb/parser/model" tidbmysql "github.com/pingcap/tidb/parser/mysql" - "github.com/shopspring/decimal" "go.uber.org/atomic" "go.uber.org/zap" @@ -345,16 +344,3 @@ func getSourceRowsForCompare(rows []*rowChange) map[string][]*dbutil.ColumnData } return rowMap } - -func genColData(v interface{}) []byte { - switch dv := v.(type) { - case []byte: - return dv - case string: - return []byte(dv) - case decimal.Decimal: - return []byte(dv.String()) - } - s := fmt.Sprintf("%v", v) - return []byte(s) -} diff --git a/dm/syncer/validate_worker_test.go b/dm/syncer/validate_worker_test.go index 7e263f5515c..291cefa944e 100644 --- a/dm/syncer/validate_worker_test.go +++ b/dm/syncer/validate_worker_test.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" - "github.com/shopspring/decimal" "github.com/stretchr/testify/require" "github.com/pingcap/tiflow/dm/dm/pb" @@ -417,16 +416,3 @@ func Test_ValidatorWorker_getSourceRowsForCompare(t *testing.T) { require.Equal(t, []byte("1"), rows["b"][0].Data) require.Equal(t, []byte("2"), rows["b"][1].Data) } - -func Test_ValidatorWorker_genColData(t *testing.T) { - res := genColData(1) - require.Equal(t, "1", string(res)) - res = genColData(1.2) - require.Equal(t, "1.2", string(res)) - res = genColData("abc") - require.Equal(t, "abc", string(res)) - res = genColData([]byte{'\x01', '\x02', '\x03'}) - require.Equal(t, "\x01\x02\x03", string(res)) - res = genColData(decimal.NewFromInt(222123123)) - require.Equal(t, "222123123", string(res)) -} From 9dd8047a23064bd5f447495968cab23a05329dc4 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Fri, 25 Feb 2022 16:40:31 +0800 Subject: [PATCH 24/36] remove worker,make pr smaller --- dm/syncer/data_validator.go | 17 +- dm/syncer/data_validator_test.go | 1 + dm/syncer/validate_worker.go | 346 ------------------------- dm/syncer/validate_worker_test.go | 418 ------------------------------ 4 files changed, 4 insertions(+), 778 deletions(-) delete mode 100644 dm/syncer/validate_worker.go delete mode 100644 dm/syncer/validate_worker_test.go diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 2d4a33bb30a..e545b27f809 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -66,6 +66,7 @@ const ( // change of table // binlog changes are clustered into table changes // the validator validates changes of table-grain at a time. +//nolint type tableChange struct { table *validateTableInfo rows map[string]*rowChange @@ -79,7 +80,8 @@ type rowChange struct { data []interface{} theType rowChangeType lastMeetTS int64 // the last meet timestamp(in seconds) - failedCnt int // failed count + //nolint + failedCnt int // failed count } // DataValidator @@ -113,7 +115,6 @@ type DataValidator struct { result pb.ProcessResult validateInterval time.Duration - workers []*validateWorker changeEventCount []atomic.Int64 workerCnt int @@ -133,7 +134,6 @@ func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *D v.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) v.workerCnt = cfg.ValidatorCfg.WorkerCount - v.workers = make([]*validateWorker, v.workerCnt) v.changeEventCount = make([]atomic.Int64, 4) v.validateInterval = validationInterval @@ -403,20 +403,9 @@ func (v *DataValidator) Stage() pb.Stage { } func (v *DataValidator) startValidateWorkers() { - v.wg.Add(v.workerCnt) - for i := 0; i < v.workerCnt; i++ { - worker := newValidateWorker(v, i) - v.workers[i] = worker - go func() { - v.wg.Done() - worker.run() - }() - } } func (v *DataValidator) dispatchRowChange(key string, row *rowChange) { - hashVal := int(utils.GenHashKey(key)) % v.workerCnt - v.workers[hashVal].rowChangeCh <- row } func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *replication.RowsEvent) error { diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index 6f2aeacb6a1..d460db22e98 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -88,6 +88,7 @@ func genSubtaskConfig(t *testing.T) *config.SubTaskConfig { return cfg } +//nolint func genDBConn(t *testing.T, db *sql.DB, cfg *config.SubTaskConfig) *dbconn.DBConn { t.Helper() baseDB := conn.NewBaseDB(db, func() {}) diff --git a/dm/syncer/validate_worker.go b/dm/syncer/validate_worker.go deleted file mode 100644 index bc8c313c6b8..00000000000 --- a/dm/syncer/validate_worker.go +++ /dev/null @@ -1,346 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package syncer - -import ( - "context" - "database/sql" - "fmt" - "math" - "strconv" - "strings" - "sync" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/tidb-tools/pkg/dbutil" - "github.com/pingcap/tidb/parser/model" - tidbmysql "github.com/pingcap/tidb/parser/mysql" - "go.uber.org/atomic" - "go.uber.org/zap" - - "github.com/pingcap/tiflow/dm/dm/config" - tcontext "github.com/pingcap/tiflow/dm/pkg/context" - "github.com/pingcap/tiflow/dm/pkg/log" - "github.com/pingcap/tiflow/dm/pkg/terror" - "github.com/pingcap/tiflow/dm/syncer/dbconn" -) - -const ( - workerChannelSize = 1000 -) - -type validateWorker struct { - cfg config.ValidatorConfig - ctx context.Context - interval time.Duration - validator *DataValidator - L log.Logger - conn *dbconn.DBConn - rowChangeCh chan *rowChange - pendingChangesMap map[string]*tableChange - pendingRowCount atomic.Int64 - sync.Mutex - - // used for test - receivedRowCount atomic.Int64 // number of rowChange received from channel - validationCount atomic.Int64 // number of successful validation -} - -func newValidateWorker(v *DataValidator, id int) *validateWorker { - return &validateWorker{ - cfg: v.cfg.ValidatorCfg, - ctx: v.ctx, - interval: v.validateInterval, - validator: v, - L: v.L, - conn: v.toDBConns[id], - rowChangeCh: make(chan *rowChange, workerChannelSize), - pendingChangesMap: make(map[string]*tableChange), - } -} - -func (vw *validateWorker) run() { - for { - select { - case change := <-vw.rowChangeCh: - // todo: limit number of pending rows - vw.updateRowChange(change) - vw.receivedRowCount.Inc() - case <-vw.ctx.Done(): - return - case <-time.After(vw.interval): - err := vw.validateTableChange() - if err != nil { - // todo: better error handling - vw.validator.errChan <- terror.Annotate(err, "failed to validate table change") - return - } - vw.validationCount.Inc() - } - } -} - -func (vw *validateWorker) updateRowChange(row *rowChange) { - // cluster using target table - fullTableName := row.table.Target.String() - change := vw.pendingChangesMap[fullTableName] - if change == nil { - // no change of this table - change = &tableChange{ - table: row.table, - rows: make(map[string]*rowChange), - } - vw.pendingChangesMap[fullTableName] = change - } - if val, ok := change.rows[row.key]; ok { - val.data = row.data - val.theType = row.theType - val.lastMeetTS = row.lastMeetTS - val.failedCnt = 0 // clear failed count - } else { - change.rows[row.key] = row - vw.pendingRowCount.Inc() - } -} - -func (vw *validateWorker) validateTableChange() error { - failedChanges := make(map[string]*tableChange) - var failedRowCnt int64 - for k, val := range vw.pendingChangesMap { - var insertUpdateChanges, deleteChanges []*rowChange - for _, r := range val.rows { - if r.theType == rowDeleted { - deleteChanges = append(deleteChanges, r) - } else { - insertUpdateChanges = append(insertUpdateChanges, r) - } - } - rows := make(map[string]*rowChange) - if len(insertUpdateChanges) > 0 { - // todo: limit number of validated rows - failedRows, err := vw.validateRowChanges(val.table, insertUpdateChanges, false) - if err != nil { - return err - } - for _, pk := range failedRows { - rows[pk] = val.rows[pk] - rows[pk].failedCnt++ - } - } - if len(deleteChanges) > 0 { - // todo: limit number of validated rows - failedRows, err := vw.validateRowChanges(val.table, deleteChanges, true) - if err != nil { - return err - } - for _, pk := range failedRows { - rows[pk] = val.rows[pk] - rows[pk].failedCnt++ - } - } - if len(rows) > 0 { - failedChanges[k] = &tableChange{ - table: val.table, - rows: rows, - } - failedRowCnt += int64(len(rows)) - } - } - vw.pendingRowCount.Store(failedRowCnt) - vw.pendingChangesMap = failedChanges - return nil -} - -func (vw *validateWorker) validateRowChanges(table *validateTableInfo, rows []*rowChange, deleteChange bool) ([]string, error) { - pkValues := make([][]string, 0, len(rows)) - for _, r := range rows { - pkValues = append(pkValues, r.pkValues) - } - colCnt := len(rows[0].data) - cond := &Cond{ - Table: table, - ColumnCnt: colCnt, - PkValues: pkValues, - } - var failedRows []string - var err error - if deleteChange { - failedRows, err = vw.validateDeletedRows(cond) - } else { - failedRows, err = vw.validateInsertAndUpdateRows(rows, cond) - } - if err != nil { - vw.L.Warn("fail to validate row changes of table", zap.Error(err)) - return nil, err - } - return failedRows, nil -} - -func (vw *validateWorker) validateDeletedRows(cond *Cond) ([]string, error) { - targetRows, err := vw.getTargetRows(cond) - if err != nil { - return []string{}, err - } - - failedRows := make([]string, 0, len(targetRows)) - for key := range targetRows { - failedRows = append(failedRows, key) - } - return failedRows, nil -} - -func (vw *validateWorker) validateInsertAndUpdateRows(rows []*rowChange, cond *Cond) ([]string, error) { - var failedRows []string - sourceRows := getSourceRowsForCompare(rows) - targetRows, err := vw.getTargetRows(cond) - if err != nil { - return nil, err - } - - if len(targetRows) > len(sourceRows) { - // if this happens, downstream should have removed the primary key - vw.L.Debug("more data on downstream, may come from other client") - } - - tableInfo := cond.Table.Info - for key, sourceRow := range sourceRows { - targetRow, ok := targetRows[key] - if !ok { - failedRows = append(failedRows, key) - continue - } - - eq, err := vw.compareData(sourceRow, targetRow, tableInfo.Columns[:cond.ColumnCnt]) - if err != nil { - return nil, err - } - if !eq { - failedRows = append(failedRows, key) - } - } - return failedRows, nil -} - -func (vw *validateWorker) compareData(sourceData, targetData []*dbutil.ColumnData, columns []*model.ColumnInfo) (bool, error) { - for i, column := range columns { - data1, data2 := sourceData[i], targetData[i] - if data1.IsNull != data2.IsNull { - return false, nil - } - str1, str2 := string(data1.Data), string(data2.Data) - if str1 == str2 { - continue - } else if column.FieldType.Tp == tidbmysql.TypeFloat || column.FieldType.Tp == tidbmysql.TypeDouble { - // source and target data have different precision? - num1, err1 := strconv.ParseFloat(str1, 64) - num2, err2 := strconv.ParseFloat(str2, 64) - if err1 != nil || err2 != nil { - // should not happen - return false, errors.Errorf("convert %s, %s to float failed, err1: %v, err2: %v", str1, str2, err1, err2) - } - if math.Abs(num1-num2) <= 1e-6 { - continue - } - } - return false, nil - } - - return true, nil -} - -func (vw *validateWorker) getTargetRows(cond *Cond) (map[string][]*dbutil.ColumnData, error) { - tctx := tcontext.NewContext(vw.ctx, log.L()) - fullTableName := cond.Table.Target.String() - pkColumnNames := make([]string, 0, len(cond.Table.pkIndices)) - for i := range cond.Table.pkIndices { - pkColumnNames = append(pkColumnNames, cond.Table.Info.Columns[i].Name.O) - } - columnNames := make([]string, 0, cond.ColumnCnt) - for i := 0; i < cond.ColumnCnt; i++ { - col := cond.Table.Info.Columns[i] - columnNames = append(columnNames, dbutil.ColumnName(col.Name.O)) - } - columns := strings.Join(columnNames, ", ") - rowsQuery := fmt.Sprintf("SELECT /*!40001 SQL_NO_CACHE */ %s FROM %s WHERE %s ORDER BY %s", - columns, fullTableName, cond.GetWhere(), strings.Join(pkColumnNames, ",")) - rows, err := vw.conn.QuerySQL(tctx, rowsQuery, cond.GetArgs()...) - if err != nil { - return nil, err - } - defer rows.Close() - - result := make(map[string][]*dbutil.ColumnData) - for rows.Next() { - rowData, err := scanRow(rows) - if err != nil { - return nil, err - } - pkVals := make([]string, 0, len(cond.Table.pkIndices)) - for _, idx := range cond.Table.pkIndices { - colVal := genColData(rowData[idx].Data) - pkVals = append(pkVals, string(colVal)) - } - pk := genRowKey(pkVals) - result[pk] = rowData - } - return result, rows.Err() -} - -func scanRow(rows *sql.Rows) ([]*dbutil.ColumnData, error) { - cols, err := rows.Columns() - if err != nil { - return nil, errors.Trace(err) - } - - colVals := make([][]byte, len(cols)) - colValsI := make([]interface{}, len(colVals)) - for i := range colValsI { - colValsI[i] = &colVals[i] - } - - err = rows.Scan(colValsI...) - if err != nil { - return nil, errors.Trace(err) - } - - result := make([]*dbutil.ColumnData, len(cols)) - for i := range colVals { - result[i] = &dbutil.ColumnData{ - Data: colVals[i], - IsNull: colVals[i] == nil, - } - } - - return result, nil -} - -func getSourceRowsForCompare(rows []*rowChange) map[string][]*dbutil.ColumnData { - rowMap := make(map[string][]*dbutil.ColumnData, len(rows)) - for _, r := range rows { - colValues := make([]*dbutil.ColumnData, len(r.data)) - for i := range r.data { - var colData []byte - if r.data[i] != nil { - colData = genColData(r.data[i]) - } - colValues[i] = &dbutil.ColumnData{ - Data: colData, - IsNull: r.data[i] == nil, - } - } - rowMap[r.key] = colValues - } - return rowMap -} diff --git a/dm/syncer/validate_worker_test.go b/dm/syncer/validate_worker_test.go deleted file mode 100644 index 291cefa944e..00000000000 --- a/dm/syncer/validate_worker_test.go +++ /dev/null @@ -1,418 +0,0 @@ -// Copyright 2022 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package syncer - -import ( - "context" - "database/sql/driver" - "sync" - "testing" - "time" - - "github.com/DATA-DOG/go-sqlmock" - "github.com/pingcap/errors" - "github.com/pingcap/tidb-tools/pkg/dbutil" - "github.com/pingcap/tidb-tools/pkg/filter" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/parser/types" - "github.com/stretchr/testify/require" - - "github.com/pingcap/tiflow/dm/dm/pb" - "github.com/pingcap/tiflow/dm/pkg/conn" - "github.com/pingcap/tiflow/dm/pkg/utils" -) - -// split into 3 cases, since it may be unstable when put together. -func Test_ValidatorWorker_run_insert_update(t *testing.T) { - tbl1 := filter.Table{Schema: "test", Name: "tbl1"} - tableInfo1 := genValidateTableInfo(t, tbl1.Schema, tbl1.Name, - "create table tbl1(a int primary key, b varchar(100))") - - cfg := genSubtaskConfig(t) - _, mock, err := conn.InitMockDBFull() - require.NoError(t, err) - defer func() { - conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} - }() - syncerObj := NewSyncer(cfg, nil, nil) - validator := NewContinuousDataValidator(cfg, syncerObj) - validator.Start(pb.Stage_Stopped) - defer validator.cancel() - - // insert & update same table, one success, one fail - worker := newValidateWorker(validator, 0) - var wg sync.WaitGroup - wg.Add(1) - go func() { - wg.Done() - worker.run() - }() - worker.rowChangeCh <- &rowChange{ - table: tableInfo1, - key: "1", - pkValues: []string{"1"}, - data: []interface{}{1, "a"}, - theType: rowInsert, - lastMeetTS: time.Now().Unix(), - } - worker.rowChangeCh <- &rowChange{ - table: tableInfo1, - key: "1", - pkValues: []string{"1"}, - data: []interface{}{1, "b"}, - theType: rowUpdated, - lastMeetTS: time.Now().Unix(), - } - worker.rowChangeCh <- &rowChange{ - table: tableInfo1, - key: "2", - pkValues: []string{"2"}, - data: []interface{}{2, "2b"}, - theType: rowInsert, - lastMeetTS: time.Now().Unix(), - } - mock.ExpectQuery("SELECT .* FROM .*tbl1.* WHERE .*").WillReturnRows( - sqlmock.NewRows([]string{"a", "b"}).AddRow(2, "incorrect data")) - require.True(t, utils.WaitSomething(30, time.Second, func() bool { - return worker.receivedRowCount.Load() == 3 - })) - require.True(t, utils.WaitSomething(30, time.Second, func() bool { - return worker.validationCount.Load() > 0 - })) - validator.cancel() - wg.Wait() - require.Equal(t, int64(3), worker.receivedRowCount.Load()) - require.Equal(t, int64(2), worker.pendingRowCount.Load()) - require.Len(t, worker.pendingChangesMap, 1) - require.Contains(t, worker.pendingChangesMap, tbl1.String()) - require.Len(t, worker.pendingChangesMap[tbl1.String()].rows, 2) - require.Contains(t, worker.pendingChangesMap[tbl1.String()].rows, "1") - require.Equal(t, rowUpdated, worker.pendingChangesMap[tbl1.String()].rows["1"].theType) - require.Equal(t, int(worker.validationCount.Load()), worker.pendingChangesMap[tbl1.String()].rows["1"].failedCnt) - require.Contains(t, worker.pendingChangesMap[tbl1.String()].rows, "2") - require.Equal(t, rowInsert, worker.pendingChangesMap[tbl1.String()].rows["2"].theType) - require.Equal(t, int(worker.validationCount.Load()), worker.pendingChangesMap[tbl1.String()].rows["2"].failedCnt) -} - -func Test_ValidatorWorker_run_all_validated(t *testing.T) { - tbl1 := filter.Table{Schema: "test", Name: "tbl1"} - tableInfo1 := genValidateTableInfo(t, tbl1.Schema, tbl1.Name, - "create table tbl1(a int primary key, b varchar(100))") - - cfg := genSubtaskConfig(t) - _, mock, err := conn.InitMockDBFull() - require.NoError(t, err) - defer func() { - conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} - }() - syncerObj := NewSyncer(cfg, nil, nil) - validator := NewContinuousDataValidator(cfg, syncerObj) - validator.Start(pb.Stage_Stopped) - defer validator.cancel() - - // insert & update same table, one success, one fail - worker := newValidateWorker(validator, 0) - var wg sync.WaitGroup - wg.Add(1) - go func() { - wg.Done() - worker.run() - }() - worker.rowChangeCh <- &rowChange{ - table: tableInfo1, - key: "1", - pkValues: []string{"1"}, - data: []interface{}{1, "a"}, - theType: rowInsert, - lastMeetTS: time.Now().Unix(), - } - worker.rowChangeCh <- &rowChange{ - table: tableInfo1, - key: "1", - pkValues: []string{"1"}, - data: []interface{}{1, "b"}, - theType: rowUpdated, - lastMeetTS: time.Now().Unix(), - } - worker.rowChangeCh <- &rowChange{ - table: tableInfo1, - key: "2", - pkValues: []string{"2"}, - data: []interface{}{2, "2b"}, - theType: rowInsert, - lastMeetTS: time.Now().Unix(), - } - mock.ExpectQuery("SELECT .* FROM .*tbl1.* WHERE .*").WillReturnRows( - sqlmock.NewRows([]string{"a", "b"}).AddRow(1, "b").AddRow(2, "2b")) - require.True(t, utils.WaitSomething(30, time.Second, func() bool { - return worker.receivedRowCount.Load() == 3 - })) - require.True(t, utils.WaitSomething(30, time.Second, func() bool { - return worker.validationCount.Load() > 0 - })) - validator.cancel() - wg.Wait() - require.Equal(t, int64(3), worker.receivedRowCount.Load()) - require.Equal(t, int64(0), worker.pendingRowCount.Load()) - require.Len(t, worker.pendingChangesMap, 0) -} - -func Test_ValidatorWorker_run_delete(t *testing.T) { - tbl2 := filter.Table{Schema: "test", Name: "tbl2"} - tbl3 := filter.Table{Schema: "test", Name: "tbl3"} - tableInfo2 := genValidateTableInfo(t, tbl2.Schema, tbl2.Name, - "create table tbl2(a varchar(100) primary key, b varchar(100))") - tableInfo3 := genValidateTableInfo(t, tbl3.Schema, tbl3.Name, - "create table tbl3(a varchar(100) primary key, b varchar(100))") - - cfg := genSubtaskConfig(t) - _, mock, err := conn.InitMockDBFull() - require.NoError(t, err) - defer func() { - conn.DefaultDBProvider = &conn.DefaultDBProviderImpl{} - }() - syncerObj := NewSyncer(cfg, nil, nil) - validator := NewContinuousDataValidator(cfg, syncerObj) - validator.Start(pb.Stage_Stopped) - defer validator.cancel() - - worker := newValidateWorker(validator, 0) - var wg sync.WaitGroup - wg.Add(1) - go func() { - wg.Done() - worker.run() - }() - worker.rowChangeCh <- &rowChange{ - table: tableInfo2, - key: "a", - pkValues: []string{"a"}, - data: []interface{}{"a", "b"}, - theType: rowDeleted, - lastMeetTS: time.Now().Unix(), - } - worker.rowChangeCh <- &rowChange{ - table: tableInfo3, - key: "aa", - pkValues: []string{"aa"}, - data: []interface{}{"aa", "b"}, - theType: rowDeleted, - lastMeetTS: time.Now().Unix(), - } - mock.ExpectQuery("SELECT .* FROM .*tbl2.* WHERE .*").WillReturnRows( - sqlmock.NewRows([]string{"a", "b"})) - mock.ExpectQuery("SELECT .* FROM .*tbl3.* WHERE .*").WillReturnRows( - sqlmock.NewRows([]string{"a", "b"}).AddRow("aa", "b")) - - // wait all events received by worker - require.True(t, utils.WaitSomething(30, time.Second, func() bool { - return worker.receivedRowCount.Load() == 2 - })) - currCnt := worker.validationCount.Load() - require.True(t, utils.WaitSomething(30, time.Second, func() bool { - return worker.validationCount.Load() > currCnt - })) - validator.cancel() - wg.Wait() - - require.Equal(t, int64(1), worker.pendingRowCount.Load()) - require.Len(t, worker.pendingChangesMap, 1) - require.Contains(t, worker.pendingChangesMap, tbl3.String()) - require.Len(t, worker.pendingChangesMap[tbl3.String()].rows, 1) - require.Contains(t, worker.pendingChangesMap[tbl3.String()].rows, "aa") - require.Equal(t, rowDeleted, worker.pendingChangesMap[tbl3.String()].rows["aa"].theType) -} - -func Test_ValidatorWorker_compareData(t *testing.T) { - worker := validateWorker{} - eq, err := worker.compareData([]*dbutil.ColumnData{{Data: []byte("1")}}, - []*dbutil.ColumnData{{IsNull: true}}, - []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeLong}}}) - require.NoError(t, err) - require.False(t, eq) - eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("1.1")}}, - []*dbutil.ColumnData{{Data: []byte("1.x")}}, - []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeFloat}}}) - require.Error(t, err) - require.False(t, eq) - eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("1.1")}}, - []*dbutil.ColumnData{{Data: []byte("1.1000011")}}, - []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeFloat}}}) - require.NoError(t, err) - require.False(t, eq) - eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("1.1")}}, - []*dbutil.ColumnData{{Data: []byte("1.1000001")}}, - []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeFloat}}}) - require.NoError(t, err) - require.True(t, eq) - eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("1.1")}}, - []*dbutil.ColumnData{{Data: []byte("1.1000001")}}, - []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeDouble}}}) - require.NoError(t, err) - require.True(t, eq) - eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("1")}}, - []*dbutil.ColumnData{{Data: []byte("1")}}, - []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeLong}}}) - require.NoError(t, err) - require.True(t, eq) - eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("aaa")}}, - []*dbutil.ColumnData{{Data: []byte("aaa")}}, - []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeVarchar}}}) - require.NoError(t, err) - require.True(t, eq) - eq, err = worker.compareData([]*dbutil.ColumnData{{Data: []byte("\x01\x02")}}, - []*dbutil.ColumnData{{Data: []byte("\x01\x02")}}, - []*model.ColumnInfo{{FieldType: types.FieldType{Tp: mysql.TypeVarString}}}) - require.NoError(t, err) - require.True(t, eq) -} - -func Test_ValidatorWorker_getTargetRows(t *testing.T) { - type testCase struct { - schemaName string - tblName string - creatSQL string - pkValues [][]string - allCols []string - rowData [][]interface{} - querySQL string - } - testCases := []testCase{ - { - schemaName: "test1", - tblName: "tbl1", - creatSQL: `create table if not exists test1.tbl1( - a int, - b int, - c int, - primary key(a, b) - );`, - pkValues: [][]string{ - {"1", "2"}, {"3", "4"}, {"5", "6"}, - }, - allCols: []string{"a", "b", "c"}, - rowData: [][]interface{}{ - {"1", "2", "3"}, {"3", "4", "5"}, {"5", "6", "7"}, - }, - querySQL: "SELECT .* FROM .*test1.*", - }, - { - schemaName: "test2", - tblName: "tbl2", - creatSQL: `create table if not exists test2.tbl2( - a varchar(10), - other text, - b varbinary(100), - c int, - primary key(a) - );`, - pkValues: [][]string{ - {"a"}, {"b"}, {"c"}, - }, - allCols: []string{"a", "other", "b", "c"}, - rowData: [][]interface{}{ - {"a", "aaa", "\xdd\xcc", "1"}, {"b", "bbb", nil, "2"}, {"c", nil, nil, "3"}, - }, - querySQL: "SELECT .* FROM .*test2.*", - }, - } - db, mock, err := sqlmock.New() - require.NoError(t, err) - for i, tc := range testCases { - var args []driver.Value - for _, arr := range testCases[i].pkValues { - for _, val := range arr { - args = append(args, val) - } - } - dataRows := mock.NewRows(tc.allCols) - for j := range testCases[i].rowData { - var rowData []driver.Value - for _, val := range tc.rowData[j] { - rowData = append(rowData, val) - } - dataRows = dataRows.AddRow(rowData...) - } - mock.ExpectQuery(tc.querySQL).WithArgs(args...).WillReturnRows(dataRows) - cond := &Cond{ - Table: genValidateTableInfo(t, tc.schemaName, tc.tblName, tc.creatSQL), - ColumnCnt: 2, - PkValues: tc.pkValues, - } - dbConn := genDBConn(t, db, genSubtaskConfig(t)) - - worker := &validateWorker{ - ctx: context.Background(), - conn: dbConn, - } - targetRows, err2 := worker.getTargetRows(cond) - require.NoError(t, err2) - require.Equal(t, 3, len(targetRows)) - for i, pkVs := range tc.pkValues { - key := genRowKey(pkVs) - require.Contains(t, targetRows, key) - data := targetRows[key] - require.Equal(t, len(tc.rowData[i]), len(data)) - for j, val := range tc.rowData[i] { - if val == nil { - require.True(t, data[j].IsNull) - require.Nil(t, data[j].Data) - } else { - require.False(t, data[j].IsNull) - require.Equal(t, val, string(data[j].Data)) - } - } - } - } - - cond := &Cond{ - Table: genValidateTableInfo(t, "test", "tbl", "create table tbl(a int primary key)"), - ColumnCnt: 1, - PkValues: [][]string{{"1"}}, - } - worker := &validateWorker{ - ctx: context.Background(), - conn: genDBConn(t, db, genSubtaskConfig(t)), - } - - // query error - mock.ExpectQuery("SELECT .* FROM .*").WithArgs(sqlmock.AnyArg()).WillReturnError(errors.New("query")) - _, err = worker.getTargetRows(cond) - require.EqualError(t, errors.Cause(err), "query") -} - -func Test_ValidatorWorker_getSourceRowsForCompare(t *testing.T) { - rows := getSourceRowsForCompare([]*rowChange{ - { - key: "a", - data: []interface{}{ - nil, 1, - }, - }, - { - key: "b", - data: []interface{}{ - 1, 2, - }, - }, - }) - require.Len(t, rows, 2) - require.Len(t, rows["a"], 2) - require.Len(t, rows["b"], 2) - require.True(t, rows["a"][0].IsNull) - require.Equal(t, []byte("1"), rows["a"][1].Data) - require.Equal(t, []byte("1"), rows["b"][0].Data) - require.Equal(t, []byte("2"), rows["b"][1].Data) -} From 5fe4564005737704a5f3a919188b17afe7b78116 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Mon, 28 Feb 2022 10:52:34 +0800 Subject: [PATCH 25/36] add log --- dm/syncer/data_validator.go | 1 + 1 file changed, 1 insertion(+) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index e545b27f809..0db8724181b 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -286,6 +286,7 @@ func (v *DataValidator) waitSyncerSynced(currLoc binlog.Location) error { if cmp <= 0 { return nil } + v.L.Info("wait syncer synced", zap.Reflect("loc", currLoc)) v.waitSyncerTimer.Reset(checkInterval) fired = false } From 28f9d3c5ff92da4a6d1ba235e12c016c591a6b4d Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Mon, 28 Feb 2022 16:02:25 +0800 Subject: [PATCH 26/36] add defer --- dm/syncer/data_validator.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 0db8724181b..e921c3aa7b6 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -242,7 +242,7 @@ func (v *DataValidator) fillResult(err error, needLock bool) { } func (v *DataValidator) errorProcessRoutine() { - v.errProcessWg.Done() + defer v.errProcessWg.Done() for { select { case err := <-v.errChan: From ee3a4ab6e0deaea6af82580842ab76d574608fb7 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Mon, 28 Feb 2022 16:25:09 +0800 Subject: [PATCH 27/36] make err channel bigger --- dm/syncer/data_validator.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index e921c3aa7b6..ac0ae60f37a 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -129,7 +129,7 @@ func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *D syncer: syncerObj, stage: pb.Stage_Stopped, // todo: many place may put into this channel, choose a proper channel size or enhance error handling - errChan: make(chan error, 1), + errChan: make(chan error, 100), } v.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) From f8f6b1153e23883fa0898a0f0fbc180fe9486bb8 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Mon, 28 Feb 2022 16:44:49 +0800 Subject: [PATCH 28/36] fix it --- dm/tests/dmctl_basic/conf/get_task.yaml | 1 + dm/tests/import_v10x/conf/task.yaml | 1 + 2 files changed, 2 insertions(+) diff --git a/dm/tests/dmctl_basic/conf/get_task.yaml b/dm/tests/dmctl_basic/conf/get_task.yaml index 3a431dbdf37..d76b328be86 100644 --- a/dm/tests/dmctl_basic/conf/get_task.yaml +++ b/dm/tests/dmctl_basic/conf/get_task.yaml @@ -164,6 +164,7 @@ syncers: validators: validator-01: mode: none + worker-count: 4 clean-dump-file: true ansi-quotes: false remove-meta: false diff --git a/dm/tests/import_v10x/conf/task.yaml b/dm/tests/import_v10x/conf/task.yaml index 10e8f111563..ac256843270 100644 --- a/dm/tests/import_v10x/conf/task.yaml +++ b/dm/tests/import_v10x/conf/task.yaml @@ -123,6 +123,7 @@ syncers: validators: validator-01: mode: none + worker-count: 4 clean-dump-file: false ansi-quotes: false remove-meta: false From 24dc127d0ff5715da6a8cef011a79ff083c4390d Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Mon, 28 Feb 2022 21:06:46 +0800 Subject: [PATCH 29/36] revert syncer stage, use isRunning --- dm/syncer/data_validator_test.go | 8 ++++---- dm/syncer/syncer.go | 19 +++++++++---------- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index d460db22e98..5ec12881d7f 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -256,15 +256,15 @@ func Test_validator_waitSyncerRunning(t *testing.T) { validator = NewContinuousDataValidator(cfg, syncerObj) validator.Start(pb.Stage_Stopped) - syncerObj.stage.Store(int32(pb.Stage_Running)) + syncerObj.running.Store(true) require.NoError(t, validator.waitSyncerRunning()) validator = NewContinuousDataValidator(cfg, syncerObj) validator.Start(pb.Stage_Stopped) - syncerObj.stage.Store(int32(pb.Stage_Stopped)) + syncerObj.running.Store(false) go func() { time.Sleep(3 * time.Second) - syncerObj.stage.Store(int32(pb.Stage_Running)) + syncerObj.running.Store(true) }() require.NoError(t, validator.waitSyncerRunning()) } @@ -288,7 +288,7 @@ func Test_validator_doValidate(t *testing.T) { }() syncerObj := NewSyncer(cfg, nil, nil) - syncerObj.stage.Store(int32(pb.Stage_Running)) + syncerObj.running.Store(true) syncerObj.tableRouter, err = router.NewRouter(cfg.CaseSensitive, []*router.TableRule{}) require.NoError(t, err) currLoc := binlog.NewLocation(cfg.Flavor) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 75700856a44..bde89f5472f 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -172,10 +172,8 @@ type Syncer struct { exprFilterGroup *ExprFilterGroup sessCtx sessionctx.Context - // int value of pb.Stage, stage diagram: - // new --> running --> stopped --> finished - // |-> finished - stage atomic.Int32 + closed atomic.Bool + running atomic.Bool start atomic.Time lastTime atomic.Time @@ -260,7 +258,8 @@ func NewSyncer(cfg *config.SubTaskConfig, etcdClient *clientv3.Client, relay rel syncer.jobsClosed.Store(true) // not open yet syncer.waitXIDJob.Store(int64(noWait)) syncer.isTransactionEnd = true - syncer.stage.Store(int32(pb.Stage_New)) + syncer.closed.Store(false) + syncer.running.Store(false) syncer.lastBinlogSizeCount.Store(0) syncer.binlogSizeCount.Store(0) syncer.lastCount.Store(0) @@ -694,8 +693,8 @@ func (s *Syncer) Process(ctx context.Context, pr chan pb.ProcessResult) { <-newCtx.Done() // ctx or newCtx }() - s.stage.Store(int32(pb.Stage_Running)) - defer s.stage.Store(int32(pb.Stage_Stopped)) + s.running.Store(true) + defer s.running.Store(false) err := s.Run(newCtx) if err != nil { @@ -3475,11 +3474,11 @@ func (s *Syncer) route(table *filter.Table) *filter.Table { } func (s *Syncer) IsRunning() bool { - return s.stage.Load() == int32(pb.Stage_Running) + return s.running.Load() } func (s *Syncer) isClosed() bool { - return s.stage.Load() == int32(pb.Stage_Finished) + return s.closed.Load() } // Close closes syncer. @@ -3505,7 +3504,7 @@ func (s *Syncer) Close() { metrics.RemoveLabelValuesWithTaskInMetrics(s.cfg.Name) s.runWg.Wait() - s.stage.Store(int32(pb.Stage_Finished)) + s.closed.Store(true) } // Kill kill syncer without graceful. From c70c560b4db41048161be2a301fa41ef5a50db28 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Tue, 1 Mar 2022 19:52:16 +0800 Subject: [PATCH 30/36] close error chan after all possible sender goroutines stopped --- dm/syncer/data_validator.go | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index ac0ae60f37a..c6e3a376971 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -128,8 +128,6 @@ func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *D cfg: cfg, syncer: syncerObj, stage: pb.Stage_Stopped, - // todo: many place may put into this channel, choose a proper channel size or enhance error handling - errChan: make(chan error, 100), } v.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) @@ -147,6 +145,8 @@ func (v *DataValidator) initialize() error { v.ctx, v.cancel = context.WithCancel(context.Background()) v.tctx = tcontext.NewContext(v.ctx, v.L) v.result.Reset() + // todo: many place may put into this channel, choose a proper channel size or enhance error handling + v.errChan = make(chan error, 10) newCtx, cancelFunc := context.WithTimeout(v.ctx, unit.DefaultInitTimeout) defer cancelFunc() @@ -243,17 +243,12 @@ func (v *DataValidator) fillResult(err error, needLock bool) { func (v *DataValidator) errorProcessRoutine() { defer v.errProcessWg.Done() - for { - select { - case err := <-v.errChan: - v.fillResult(err, true) + for err := range v.errChan { + v.fillResult(err, true) - if errors.Cause(err) != context.Canceled { - // todo: need a better way to handle err(auto resuming on some error, etc.) - v.stopInner() - } - case <-v.ctx.Done(): - return + if errors.Cause(err) != context.Canceled { + // todo: need a better way to handle err(auto resuming on some error, etc.) + v.stopInner() } } } @@ -388,6 +383,8 @@ func (v *DataValidator) stopInner() { v.toDB.Close() v.wg.Wait() + close(v.errChan) // close error chan after all possible sender goroutines stopped + v.stage = pb.Stage_Stopped } From d374d388fa1470714890f0ba5e69697db3ad5808 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Wed, 2 Mar 2022 11:44:57 +0800 Subject: [PATCH 31/36] fix comments --- dm/syncer/data_validator.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index c6e3a376971..8f8f7f8817d 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -78,7 +78,7 @@ type rowChange struct { key string pkValues []string data []interface{} - theType rowChangeType + tp rowChangeType lastMeetTS int64 // the last meet timestamp(in seconds) //nolint failedCnt int // failed count @@ -132,7 +132,7 @@ func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *D v.L = log.With(zap.String("task", cfg.Name), zap.String("unit", "continuous validator")) v.workerCnt = cfg.ValidatorCfg.WorkerCount - v.changeEventCount = make([]atomic.Int64, 4) + v.changeEventCount = make([]atomic.Int64, 3) v.validateInterval = validationInterval v.unsupportedTable = make(map[string]string) @@ -498,7 +498,7 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re key: key, pkValues: pkValue, data: row, - theType: rowDeleted, + tp: rowDeleted, lastMeetTS: int64(header.Timestamp), }) afterRowChangeType = rowInsert @@ -508,7 +508,7 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re key: afterKey, pkValues: afterPkValue, data: afterRow, - theType: afterRowChangeType, + tp: afterRowChangeType, lastMeetTS: int64(header.Timestamp), }) } else { @@ -517,7 +517,7 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re key: key, pkValues: pkValue, data: row, - theType: changeType, + tp: changeType, lastMeetTS: int64(header.Timestamp), }) } From 2317d64b710cb724ce4009c3e2dc7e26905d37aa Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Wed, 2 Mar 2022 13:49:10 +0800 Subject: [PATCH 32/36] timer to time.after --- dm/syncer/data_validator.go | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 8f8f7f8817d..51c5dee58b2 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -120,7 +120,6 @@ type DataValidator struct { // such as table without primary key unsupportedTable map[string]string - waitSyncerTimer *time.Timer } func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *DataValidator { @@ -136,7 +135,6 @@ func NewContinuousDataValidator(cfg *config.SubTaskConfig, syncerObj *Syncer) *D v.validateInterval = validationInterval v.unsupportedTable = make(map[string]string) - v.waitSyncerTimer = utils.NewStoppedTimer() return v } @@ -260,30 +258,17 @@ func (v *DataValidator) waitSyncerSynced(currLoc binlog.Location) error { return nil } - fired := false - v.waitSyncerTimer.Reset(checkInterval) - defer func() { - if !fired { - if !v.waitSyncerTimer.Stop() { - <-v.waitSyncerTimer.C - } - } - }() - for { select { case <-v.ctx.Done(): return v.ctx.Err() - case <-v.waitSyncerTimer.C: - fired = true + case <-time.After(checkInterval): syncLoc = v.syncer.checkpoint.FlushedGlobalPoint() cmp = binlog.CompareLocation(currLoc, syncLoc, v.cfg.EnableGTID) if cmp <= 0 { return nil } v.L.Info("wait syncer synced", zap.Reflect("loc", currLoc)) - v.waitSyncerTimer.Reset(checkInterval) - fired = false } } } From d968aa99adeb07e920869a4b01d212195172924b Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Wed, 2 Mar 2022 13:58:11 +0800 Subject: [PATCH 33/36] fix comments --- dm/syncer/data_validator.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 51c5dee58b2..d04b40ca5ab 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -296,7 +296,8 @@ func (v *DataValidator) doValidate() { return } - // todo: syncer may change replication location(start from timestamp, sharding resync), how validator react? + // todo: if validator starts on task start, we need to make sure the location we got is the init location of syncer + // todo: right now, there's change they have a gap location := v.syncer.checkpoint.FlushedGlobalPoint() if v.streamerController.IsClosed() { err := v.streamerController.Start(v.tctx, location) From a4b807dd503b584890df904e5a2667c136ddb301 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Wed, 2 Mar 2022 14:24:33 +0800 Subject: [PATCH 34/36] fix comments --- dm/syncer/data_validator.go | 11 ++++++----- dm/syncer/data_validator_test.go | 16 ++++++++-------- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index d04b40ca5ab..ed0d049d247 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser/model" - "github.com/shopspring/decimal" "go.uber.org/atomic" "go.uber.org/zap" @@ -52,7 +51,7 @@ type validateTableInfo struct { Info *model.TableInfo PrimaryKey *model.IndexInfo Target *filter.Table // target table after route - pkIndices []int + pkIndices []int // TODO: can we use offset of in indexColumn? may remove this field in that case } type rowChangeType int @@ -73,6 +72,8 @@ type tableChange struct { } // change of a row. +// todo: this struct and some logic on it may reuse RowChange in pkg/sqlmodel +// todo: maybe we can use reuse it later. type rowChange struct { table *validateTableInfo key string @@ -423,6 +424,8 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re primaryIdx = idx } } + // todo: PKIsHandle = true when table has primary key like "id int primary key CLUSTERED", since schema-tracker(we get from it) + // todo: only use downstream DDL when the task is incremental only, will support this case later. if primaryIdx == nil { // todo: for table without primary index, need to record in the failed table, will add it later together with checkpoint v.unsupportedTable[fullTableName] = "without primary key" @@ -478,7 +481,7 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re } afterKey := genRowKey(afterPkValue) if afterKey != key { - // convert to delete and insert + // TODO: may reuse IsIdentityUpdated/SplitUpdate of RowChange in pkg/sqlmodel v.dispatchRowChange(key, &rowChange{ table: table, key: key, @@ -534,8 +537,6 @@ func genColData(v interface{}) []byte { return dv case string: return []byte(dv) - case decimal.Decimal: - return []byte(dv.String()) } s := fmt.Sprintf("%v", v) return []byte(s) diff --git a/dm/syncer/data_validator_test.go b/dm/syncer/data_validator_test.go index 5ec12881d7f..8656b3e9ff4 100644 --- a/dm/syncer/data_validator_test.go +++ b/dm/syncer/data_validator_test.go @@ -100,7 +100,7 @@ func genDBConn(t *testing.T, db *sql.DB, cfg *config.SubTaskConfig) *dbconn.DBCo } } -func Test_validator_StartStop(t *testing.T) { +func TestValidatorStartStop(t *testing.T) { cfg := genSubtaskConfig(t) syncerObj := NewSyncer(cfg, nil, nil) @@ -147,7 +147,7 @@ func Test_validator_StartStop(t *testing.T) { validator.Stop() } -func Test_validator_fillResult(t *testing.T) { +func TestValidatorFillResult(t *testing.T) { cfg := genSubtaskConfig(t) syncerObj := NewSyncer(cfg, nil, nil) _, _, err := conn.InitMockDBFull() @@ -168,7 +168,7 @@ func Test_validator_fillResult(t *testing.T) { require.Len(t, validator.result.Errors, 2) } -func Test_validator_errorProcessRoutine(t *testing.T) { +func TestValidatorErrorProcessRoutine(t *testing.T) { cfg := genSubtaskConfig(t) syncerObj := NewSyncer(cfg, nil, nil) _, _, err := conn.InitMockDBFull() @@ -203,7 +203,7 @@ func (c *mockedCheckPointForValidator) FlushedGlobalPoint() binlog.Location { return c.nextLoc } -func Test_validator_waitSyncerSynced(t *testing.T) { +func TestValidatorWaitSyncerSynced(t *testing.T) { cfg := genSubtaskConfig(t) syncerObj := NewSyncer(cfg, nil, nil) _, _, err := conn.InitMockDBFull() @@ -240,7 +240,7 @@ func Test_validator_waitSyncerSynced(t *testing.T) { require.NoError(t, validator.waitSyncerSynced(currLoc)) } -func Test_validator_waitSyncerRunning(t *testing.T) { +func TestValidatorWaitSyncerRunning(t *testing.T) { cfg := genSubtaskConfig(t) syncerObj := NewSyncer(cfg, nil, nil) _, _, err := conn.InitMockDBFull() @@ -269,7 +269,7 @@ func Test_validator_waitSyncerRunning(t *testing.T) { require.NoError(t, validator.waitSyncerRunning()) } -func Test_validator_doValidate(t *testing.T) { +func TestValidatorDoValidate(t *testing.T) { var ( schemaName = "test" tableName = "tbl" @@ -434,7 +434,7 @@ func Test_validator_doValidate(t *testing.T) { require.Contains(t, validator.unsupportedTable, ft.String()) } -func Test_validator_getRowChangeType(t *testing.T) { +func TestValidatorGetRowChangeType(t *testing.T) { require.Equal(t, rowInsert, getRowChangeType(replication.WRITE_ROWS_EVENTv0)) require.Equal(t, rowInsert, getRowChangeType(replication.WRITE_ROWS_EVENTv1)) require.Equal(t, rowInsert, getRowChangeType(replication.WRITE_ROWS_EVENTv2)) @@ -446,7 +446,7 @@ func Test_validator_getRowChangeType(t *testing.T) { require.Equal(t, rowDeleted, getRowChangeType(replication.DELETE_ROWS_EVENTv2)) } -func Test_validator_genColData(t *testing.T) { +func TestValidatorGenColData(t *testing.T) { res := genColData(1) require.Equal(t, "1", string(res)) res = genColData(1.2) From 0c75d9f4a2eafc7c60a563a544c1dec087790afe Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 3 Mar 2022 10:33:50 +0800 Subject: [PATCH 35/36] fix comments --- dm/syncer/data_validator.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index ed0d049d247..77b1c489437 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -300,6 +300,7 @@ func (v *DataValidator) doValidate() { // todo: if validator starts on task start, we need to make sure the location we got is the init location of syncer // todo: right now, there's change they have a gap location := v.syncer.checkpoint.FlushedGlobalPoint() + // it's for test, some fields in streamerController is mocked, cannot call Start if v.streamerController.IsClosed() { err := v.streamerController.Start(v.tctx, location) if err != nil { @@ -403,6 +404,14 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re return nil } + needSkip, err := v.syncer.skipRowsEvent(sourceTable, header.EventType) + if err != nil { + return err + } + if needSkip { + return nil + } + targetTable := v.syncer.route(sourceTable) tableInfo, err := v.syncer.schemaTracker.GetTableInfo(sourceTable) if err != nil { From 6277f069e3e9906cf37678ab947f6c53924e2d33 Mon Sep 17 00:00:00 2001 From: D3Hunter Date: Thu, 3 Mar 2022 14:45:47 +0800 Subject: [PATCH 36/36] fix comments --- dm/syncer/data_validator.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dm/syncer/data_validator.go b/dm/syncer/data_validator.go index 77b1c489437..155076bf02f 100644 --- a/dm/syncer/data_validator.go +++ b/dm/syncer/data_validator.go @@ -413,6 +413,8 @@ func (v *DataValidator) processRowsEvent(header *replication.EventHeader, ev *re } targetTable := v.syncer.route(sourceTable) + // todo: syncer will change schema in schemaTracker, will there be data race? + // todo: what if table is dropped while validator falls behind? tableInfo, err := v.syncer.schemaTracker.GetTableInfo(sourceTable) if err != nil { if schema.IsTableNotExists(err) {