From aa34581dc38fd615dbfaea5144f788c660863524 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Thu, 29 Dec 2022 19:23:05 +0800 Subject: [PATCH 01/20] print information about the skipped table in summary --- sync_diff_inspector/diff.go | 38 ++++++-- sync_diff_inspector/progress/progress.go | 6 +- sync_diff_inspector/report/report.go | 34 +++++--- sync_diff_inspector/source/chunks_iter.go | 2 +- .../source/common/table_diff.go | 5 ++ sync_diff_inspector/source/mysql_shard.go | 87 +++++++++++-------- sync_diff_inspector/source/source.go | 33 +++++-- sync_diff_inspector/source/source_test.go | 32 ++++--- sync_diff_inspector/source/tidb.go | 13 +-- sync_diff_inspector/utils/utils.go | 2 +- .../table_skip/config_base.toml | 46 ++++++++++ tests/sync_diff_inspector/table_skip/data.sql | 6 ++ tests/sync_diff_inspector/table_skip/run.sh | 29 +++++++ 13 files changed, 242 insertions(+), 91 deletions(-) create mode 100644 tests/sync_diff_inspector/table_skip/config_base.toml create mode 100644 tests/sync_diff_inspector/table_skip/data.sql create mode 100644 tests/sync_diff_inspector/table_skip/run.sh diff --git a/sync_diff_inspector/diff.go b/sync_diff_inspector/diff.go index e7c04b651..d2f30f6a9 100644 --- a/sync_diff_inspector/diff.go +++ b/sync_diff_inspector/diff.go @@ -297,12 +297,19 @@ func (df *Diff) StructEqual(ctx context.Context) error { tableIndex = df.startRange.ChunkRange.Index.TableIndex } for ; tableIndex < len(tables); tableIndex++ { - isEqual, isSkip, err := df.compareStruct(ctx, tableIndex) - if err != nil { - return errors.Trace(err) + var isEqual, isSkip bool + isAllExist := tables[tableIndex].NeedSkippedTable + if isAllExist == 0 { + var err error + isEqual, isSkip, err = df.compareStruct(ctx, tableIndex) + if err != nil { + return errors.Trace(err) + } + } else { + isEqual, isSkip = false, true } - progress.RegisterTable(dbutil.TableName(tables[tableIndex].Schema, tables[tableIndex].Table), !isEqual, isSkip) - df.report.SetTableStructCheckResult(tables[tableIndex].Schema, tables[tableIndex].Table, isEqual, isSkip) + progress.RegisterTable(dbutil.TableName(tables[tableIndex].Schema, tables[tableIndex].Table), !isEqual, isSkip, isAllExist) + df.report.SetTableStructCheckResult(tables[tableIndex].Schema, tables[tableIndex].Table, isEqual, isSkip, isAllExist) } return nil } @@ -411,12 +418,28 @@ func (df *Diff) consume(ctx context.Context, rangeInfo *splitter.RangeInfo) bool node: rangeInfo.ToNode(), } defer func() { df.sqlCh <- dml }() + tableDiff := df.downstream.GetTables()[rangeInfo.GetTableIndex()] + schema, table := tableDiff.Schema, tableDiff.Table + id := rangeInfo.ChunkRange.Index if rangeInfo.ChunkRange.Type == chunk.Empty { dml.node.State = checkpoints.IgnoreState + // for table only exists upstream + if tableDiff.NeedSkippedTable == -1 { + upCount, _ := dbutil.GetRowCount(ctx, df.upstream.GetDB(), schema, table, "", nil) + isEqual := false + df.report.SetTableDataCheckResult(schema, table, false, int(upCount), 0, upCount, 0, id) + return isEqual + } + // for table only exists downstream + if tableDiff.NeedSkippedTable == 1 { + downCount, _ := dbutil.GetRowCount(ctx, df.downstream.GetDB(), schema, table, "", nil) + isEqual := false + df.report.SetTableDataCheckResult(schema, table, isEqual, 0, int(downCount), 0, downCount, id) + return isEqual + } return true } - tableDiff := df.downstream.GetTables()[rangeInfo.GetTableIndex()] - schema, table := tableDiff.Schema, tableDiff.Table + var state string = checkpoints.SuccessState isEqual, upCount, downCount, err := df.compareChecksumAndGetCount(ctx, rangeInfo) @@ -447,7 +470,6 @@ func (df *Diff) consume(ctx context.Context, rangeInfo *splitter.RangeInfo) bool isEqual = isDataEqual } dml.node.State = state - id := rangeInfo.ChunkRange.Index df.report.SetTableDataCheckResult(schema, table, isEqual, dml.rowAdd, dml.rowDelete, upCount, downCount, id) return isEqual } diff --git a/sync_diff_inspector/progress/progress.go b/sync_diff_inspector/progress/progress.go index 69dcc2baa..f0b689945 100644 --- a/sync_diff_inspector/progress/progress.go +++ b/sync_diff_inspector/progress/progress.go @@ -127,7 +127,7 @@ func (tpp *TableProgressPrinter) UpdateTotal(name string, total int, stopUpdate } } -func (tpp *TableProgressPrinter) RegisterTable(name string, isFailed bool, isDone bool) { +func (tpp *TableProgressPrinter) RegisterTable(name string, isFailed bool, isDone bool, isAllExist int) { var state table_state_t if isFailed { if isDone { @@ -410,9 +410,9 @@ func UpdateTotal(name string, total int, stopUpdate bool) { } } -func RegisterTable(name string, isFailed bool, isDone bool) { +func RegisterTable(name string, isFailed bool, isDone bool, isAllExist int) { if progress_ != nil { - progress_.RegisterTable(name, isFailed, isDone) + progress_.RegisterTable(name, isFailed, isDone, isAllExist) } } diff --git a/sync_diff_inspector/report/report.go b/sync_diff_inspector/report/report.go index 684e581c1..cb1f5e0a9 100644 --- a/sync_diff_inspector/report/report.go +++ b/sync_diff_inspector/report/report.go @@ -56,16 +56,16 @@ type ReportConfig struct { // TableResult saves the check result for every table. type TableResult struct { - Schema string `json:"schema"` - Table string `json:"table"` - StructEqual bool `json:"struct-equal"` - DataSkip bool `json:"data-skip"` - DataEqual bool `json:"data-equal"` - MeetError error `json:"-"` - ChunkMap map[string]*ChunkResult `json:"chunk-result"` // `ChunkMap` stores the `ChunkResult` of each chunk of the table - UpCount int64 `json:"up-count"` // `UpCount` is the number of rows in the table from upstream - DownCount int64 `json:"down-count"` // `DownCount` is the number of rows in the table from downstream - + Schema string `json:"schema"` + Table string `json:"table"` + StructEqual bool `json:"struct-equal"` + DataSkip bool `json:"data-skip"` + DataEqual bool `json:"data-equal"` + MeetError error `json:"-"` + ChunkMap map[string]*ChunkResult `json:"chunk-result"` // `ChunkMap` stores the `ChunkResult` of each chunk of the table + UpCount int64 `json:"up-count"` // `UpCount` is the number of rows in the table from upstream + DownCount int64 `json:"down-count"` // `DownCount` is the number of rows in the table from downstream + TableSkipped int `json:"table-skipped"` } // ChunkResult save the necessarily information to provide summary information @@ -131,6 +131,11 @@ func (r *Report) getDiffRows() [][]string { } diffRow := make([]string, 0) diffRow = append(diffRow, dbutil.TableName(schema, table)) + if result.TableSkipped != 0 { + diffRow = append(diffRow, "skipped") + } else { + diffRow = append(diffRow, "succeed") + } if !result.StructEqual { diffRow = append(diffRow, "false") } else { @@ -154,8 +159,8 @@ func (r *Report) CalculateTotalSize(ctx context.Context, db *sql.DB) { for schema, tableMap := range r.TableResults { for table := range tableMap { size, err := utils.GetTableSize(ctx, db, schema, table) - - if size == 0 || err != nil { + tableSkipped := r.TableResults[schema][table].TableSkipped + if (size == 0 || err != nil) && tableSkipped == 0 { log.Warn("fail to get the correct size of table, if you want to get the correct size, please analyze the corresponding tables", zap.String("table", dbutil.TableName(schema, table)), zap.Error(err)) } else { r.TotalSize += size @@ -212,7 +217,7 @@ func (r *Report) CommitSummary() error { summaryFile.WriteString("The following tables contains inconsistent data\n\n") tableString := &strings.Builder{} table := tablewriter.NewWriter(tableString) - table.SetHeader([]string{"Table", "Structure equality", "Data diff rows", "UpCount", "DownCount"}) + table.SetHeader([]string{"Table", "RESULT", "Structure equality", "Data diff rows", "UpCount", "DownCount"}) diffRows := r.getDiffRows() for _, v := range diffRows { table.Append(v) @@ -295,12 +300,13 @@ func (r *Report) Init(tableDiffs []*common.TableDiff, sourceConfig [][]byte, tar } // SetTableStructCheckResult sets the struct check result for table. -func (r *Report) SetTableStructCheckResult(schema, table string, equal bool, skip bool) { +func (r *Report) SetTableStructCheckResult(schema, table string, equal bool, skip bool, exist int) { r.Lock() defer r.Unlock() tableResult := r.TableResults[schema][table] tableResult.StructEqual = equal tableResult.DataSkip = skip + tableResult.TableSkipped = exist if !equal && r.Result != Error { r.Result = Fail } diff --git a/sync_diff_inspector/source/chunks_iter.go b/sync_diff_inspector/source/chunks_iter.go index f7246eee5..104ff3435 100644 --- a/sync_diff_inspector/source/chunks_iter.go +++ b/sync_diff_inspector/source/chunks_iter.go @@ -103,7 +103,7 @@ func (t *ChunksIterator) produceChunks(ctx context.Context, startRange *splitter for ; t.nextTableIndex < len(t.TableDiffs); t.nextTableIndex++ { curTableIndex := t.nextTableIndex // skip data-check, but still need to send a empty chunk to make checkpoint continuous - if t.TableDiffs[curTableIndex].IgnoreDataCheck { + if t.TableDiffs[curTableIndex].IgnoreDataCheck || t.TableDiffs[curTableIndex].NeedSkippedTable != 0 { pool.Apply(func() { table := t.TableDiffs[curTableIndex] progressID := dbutil.TableName(table.Schema, table.Table) diff --git a/sync_diff_inspector/source/common/table_diff.go b/sync_diff_inspector/source/common/table_diff.go index d6d1b9157..33816a38f 100644 --- a/sync_diff_inspector/source/common/table_diff.go +++ b/sync_diff_inspector/source/common/table_diff.go @@ -65,4 +65,9 @@ type TableDiff struct { Collation string `json:"collation"` ChunkSize int64 `json:"chunk-size"` + + // NeedSkippedTable = 1: the table only exists downstream, + // NeedSkippedTable = -1: the table only exists upstream, + // NeedSkippedTable = 0: the table exists both upstream and downstream. + NeedSkippedTable int `json:"-"` } diff --git a/sync_diff_inspector/source/mysql_shard.go b/sync_diff_inspector/source/mysql_shard.go index 004b8195e..428e39973 100644 --- a/sync_diff_inspector/source/mysql_shard.go +++ b/sync_diff_inspector/source/mysql_shard.go @@ -98,43 +98,50 @@ func (s *MySQLSources) GetCountAndCrc32(ctx context.Context, tableRange *splitte table := s.tableDiffs[tableRange.GetTableIndex()] chunk := tableRange.GetChunk() - matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) - infoCh := make(chan *ChecksumInfo, len(s.sourceTablesMap)) - - for _, ms := range matchSources { - go func(ms *common.TableShardSource) { - count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, table.Info, chunk.Where, chunk.Args) - infoCh <- &ChecksumInfo{ - Checksum: checksum, - Count: count, - Err: err, + // for tables that do not exist upstream or downstream + if table.NeedSkippedTable != 0 { + return &ChecksumInfo{ + Count: 0, + } + } else { + matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) + infoCh := make(chan *ChecksumInfo, len(s.sourceTablesMap)) + + for _, ms := range matchSources { + go func(ms *common.TableShardSource) { + count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, table.Info, chunk.Where, chunk.Args) + infoCh <- &ChecksumInfo{ + Checksum: checksum, + Count: count, + Err: err, + } + }(ms) + } + defer close(infoCh) + + var ( + err error + totalCount int64 + totalChecksum int64 + ) + + for range matchSources { + info := <-infoCh + // catch the first error + if err == nil && info.Err != nil { + err = info.Err } - }(ms) - } - defer close(infoCh) - - var ( - err error - totalCount int64 - totalChecksum int64 - ) - - for range matchSources { - info := <-infoCh - // catch the first error - if err == nil && info.Err != nil { - err = info.Err + totalCount += info.Count + totalChecksum ^= info.Checksum } - totalCount += info.Count - totalChecksum ^= info.Checksum - } - cost := time.Since(beginTime) - return &ChecksumInfo{ - Checksum: totalChecksum, - Count: totalCount, - Err: err, - Cost: cost, + cost := time.Since(beginTime) + return &ChecksumInfo{ + Checksum: totalChecksum, + Count: totalCount, + Err: err, + Cost: cost, + } } } @@ -162,6 +169,10 @@ func (s *MySQLSources) GetRowsIterator(ctx context.Context, tableRange *splitter sourceRows := make(map[int]*sql.Rows) table := s.tableDiffs[tableRange.GetTableIndex()] + // for tables that do not exist upstream or downstream + if table.NeedSkippedTable != 0 { + return nil, nil + } matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) var rowsQuery string @@ -223,6 +234,10 @@ func (s *MySQLSources) GetSnapshot() string { func (s *MySQLSources) GetSourceStructInfo(ctx context.Context, tableIndex int) ([]*model.TableInfo, error) { tableDiff := s.GetTables()[tableIndex] + // for tables that do not exist upstream or downstream + if tableDiff.NeedSkippedTable != 0 { + return nil, nil + } tableSources := getMatchedSourcesForTable(s.sourceTablesMap, tableDiff) sourceTableInfos := make([]*model.TableInfo, len(tableSources)) for i, tableSource := range tableSources { @@ -355,9 +370,7 @@ func NewMySQLSources(ctx context.Context, tableDiffs []*common.TableDiff, ds []* } - if err := checkTableMatched(targetUniqueTableMap, sourceTablesAfterRoute); err != nil { - return nil, errors.Annotatef(err, "please make sure the filter is correct.") - } + tableDiffs = checkTableMatched(tableDiffs, targetUniqueTableMap, sourceTablesAfterRoute) mss := &MySQLSources{ tableDiffs: tableDiffs, diff --git a/sync_diff_inspector/source/source.go b/sync_diff_inspector/source/source.go index 3ca96c57f..7190bcf39 100644 --- a/sync_diff_inspector/source/source.go +++ b/sync_diff_inspector/source/source.go @@ -208,7 +208,7 @@ func NewSources(ctx context.Context, cfg *config.Config) (downstream Source, ups if err != nil { return nil, nil, errors.Annotate(err, "from upstream") } - downstream, err = buildSourceFromCfg(ctx, tableDiffs, mysqlConnCount, bucketSpliterPool, cfg.Task.TargetCheckTables, cfg.Task.TargetInstance) + downstream, err = buildSourceFromCfg(ctx, upstream.GetTables(), mysqlConnCount, bucketSpliterPool, cfg.Task.TargetCheckTables, cfg.Task.TargetInstance) if err != nil { return nil, nil, errors.Annotate(err, "from downstream") } @@ -375,21 +375,42 @@ type RangeIterator interface { Close() } -func checkTableMatched(targetMap map[string]struct{}, sourceMap map[string]struct{}) error { +func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]struct{}, sourceMap map[string]struct{}) []*common.TableDiff { // check target exists but source not found for tableDiff := range targetMap { // target table have all passed in tableFilter if _, ok := sourceMap[tableDiff]; !ok { - return errors.Errorf("the source has no table to be compared. target-table is `%s`", tableDiff) + index := getIndexByUniqueID(tableDiffs, tableDiff) + if tableDiffs[index].NeedSkippedTable == 0 { + tableDiffs[index].NeedSkippedTable = 1 + log.Info("the source has no table to be compared", zap.String("target-table", tableDiff)) + } } } // check source exists but target not found for tableDiff := range sourceMap { // need check source table have passd in tableFilter here if _, ok := targetMap[tableDiff]; !ok { - return errors.Errorf("the target has no table to be compared. source-table is `%s`", tableDiff) + slice := strings.Split(strings.Replace(tableDiff, "`", "", -1), ".") + tableDiffs = append(tableDiffs, &common.TableDiff{ + Schema: slice[0], + Table: slice[1], + NeedSkippedTable: -1, + }) + log.Info("the target has no table to be compared", zap.String("source-table", tableDiff)) } } - log.Info("table match check passed!!") - return nil + log.Info("table match check finished") + return tableDiffs +} + +// Get the index of table in tableDiffs by uniqueID:`schema`.`table` +func getIndexByUniqueID(tableDiffs []*common.TableDiff, uniqueID string) int { + for i := 0; i < len(tableDiffs); i++ { + tableUniqueID := utils.UniqueID(tableDiffs[i].Schema, tableDiffs[i].Table) + if tableUniqueID == uniqueID { + return i + } + } + return 0 } diff --git a/sync_diff_inspector/source/source_test.go b/sync_diff_inspector/source/source_test.go index 1bcc9fb01..7ad151530 100644 --- a/sync_diff_inspector/source/source_test.go +++ b/sync_diff_inspector/source/source_test.go @@ -910,20 +910,32 @@ func TestInitTables(t *testing.T) { } func TestCheckTableMatched(t *testing.T) { + var tableDiffs []*common.TableDiff + tableDiffs = append(tableDiffs, &common.TableDiff{ + Schema: "test", + Table: "t1", + }) + tableDiffs = append(tableDiffs, &common.TableDiff{ + Schema: "test", + Table: "t2", + }) + tmap := make(map[string]struct{}) smap := make(map[string]struct{}) - tmap["1"] = struct{}{} - tmap["2"] = struct{}{} + smap["`test`.`t1`"] = struct{}{} + smap["`test`.`t2`"] = struct{}{} - smap["1"] = struct{}{} - smap["2"] = struct{}{} - require.NoError(t, checkTableMatched(tmap, smap)) + tmap["`test`.`t1`"] = struct{}{} + tmap["`test`.`t2`"] = struct{}{} - delete(smap, "1") - require.Contains(t, checkTableMatched(tmap, smap).Error(), "the source has no table to be compared. target-table") + tables := checkTableMatched(tableDiffs, tmap, smap) + require.Equal(t, 0, tables[0].NeedSkippedTable) + require.Equal(t, 0, tables[1].NeedSkippedTable) - delete(tmap, "1") - smap["1"] = struct{}{} - require.Contains(t, checkTableMatched(tmap, smap).Error(), "the target has no table to be compared. source-table") + delete(smap, "`test`.`t2`") + smap["`test`.`t3`"] = struct{}{} + tables = checkTableMatched(tableDiffs, tmap, smap) + require.Equal(t, 1, tables[1].NeedSkippedTable) + require.Equal(t, -1, tables[2].NeedSkippedTable) } diff --git a/sync_diff_inspector/source/tidb.go b/sync_diff_inspector/source/tidb.go index 86ebdf826..9a67ad8aa 100644 --- a/sync_diff_inspector/source/tidb.go +++ b/sync_diff_inspector/source/tidb.go @@ -194,7 +194,7 @@ func (s *TiDBSource) GetSnapshot() string { return s.snapshot } -func getSourceTableMap(ctx context.Context, tableDiffs []*common.TableDiff, ds *config.DataSource, f tableFilter.Filter) (map[string]*common.TableSource, error) { +func NewTiDBSource(ctx context.Context, tableDiffs []*common.TableDiff, ds *config.DataSource, bucketSpliterPool *utils.WorkerPool, f tableFilter.Filter) (Source, error) { sourceTableMap := make(map[string]*common.TableSource) log.Info("find router for tidb source") // we should get the real table name @@ -253,17 +253,8 @@ func getSourceTableMap(ctx context.Context, tableDiffs []*common.TableDiff, ds * } } - if err = checkTableMatched(targetUniqueTableMap, sourceTablesAfterRoute); err != nil { - return nil, errors.Annotatef(err, "please make sure the filter is correct.") - } - return sourceTableMap, nil -} + tableDiffs = checkTableMatched(tableDiffs, targetUniqueTableMap, sourceTablesAfterRoute) -func NewTiDBSource(ctx context.Context, tableDiffs []*common.TableDiff, ds *config.DataSource, bucketSpliterPool *utils.WorkerPool, f tableFilter.Filter) (Source, error) { - sourceTableMap, err := getSourceTableMap(ctx, tableDiffs, ds, f) - if err != nil { - return nil, errors.Trace(err) - } ts := &TiDBSource{ tableDiffs: tableDiffs, sourceTableMap: sourceTableMap, diff --git a/sync_diff_inspector/utils/utils.go b/sync_diff_inspector/utils/utils.go index ed27b0917..68e0ad6de 100644 --- a/sync_diff_inspector/utils/utils.go +++ b/sync_diff_inspector/utils/utils.go @@ -909,7 +909,7 @@ func ResetColumns(tableInfo *model.TableInfo, columns []string) (*model.TableInf return tableInfo, hasTimeStampType } -// UniqueID returns `schema:table` +// UniqueID returns `schema`.`table` func UniqueID(schema string, table string) string { // QuoteSchema quotes a full table name return fmt.Sprintf("`%s`.`%s`", EscapeName(schema), EscapeName(table)) diff --git a/tests/sync_diff_inspector/table_skip/config_base.toml b/tests/sync_diff_inspector/table_skip/config_base.toml new file mode 100644 index 000000000..7130eddda --- /dev/null +++ b/tests/sync_diff_inspector/table_skip/config_base.toml @@ -0,0 +1,46 @@ +# Diff Configuration. + +######################### Global config ######################### + +# how many goroutines are created to check data +check-thread-count = 4 + +# set false if just want compare data by checksum, will skip select data when checksum is not equal. +# set true if want compare all different rows, will slow down the total compare time. +export-fix-sql = true + +# ignore check table's data +check-struct-only = false + +######################### Databases config ######################### +[data-sources] +[data-sources.mysql1] + host = "127.0.0.1"#MYSQL_HOST + port = 3306#MYSQL_PORT + user = "root" + password = "123456" + # remove comment if use tidb's snapshot data + # snapshot = "2016-10-08 16:45:26" + +[data-sources.tidb] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + # remove comment if use tidb's snapshot data + # snapshot = "2016-10-08 16:45:26" + +######################### Task config ######################### +[task] + # 1 fix sql: fix-target-TIDB1.sql + # 2 log: sync-diff.log + # 3 summary: summary.txt + # 4 checkpoint: a dir + output-dir = "/tmp/tidb_tools_test/sync_diff_inspector/output" + + source-instances = ["mysql1"] + + target-instance = "tidb" + + # tables need to check. + target-check-tables = ["diff_test.t*"] \ No newline at end of file diff --git a/tests/sync_diff_inspector/table_skip/data.sql b/tests/sync_diff_inspector/table_skip/data.sql new file mode 100644 index 000000000..3645cdb7d --- /dev/null +++ b/tests/sync_diff_inspector/table_skip/data.sql @@ -0,0 +1,6 @@ +drop database if exit diff_test; +create database diff_test; +create table diff_test.t0 (a int, b int, primary key(a)); +create table diff_test.t1 (a int, b int, primary key(a)); +insert into diff_test.t0 values (1,1) +insert into diff_test.t1 values (1,1) \ No newline at end of file diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh new file mode 100644 index 000000000..324c5c092 --- /dev/null +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -0,0 +1,29 @@ +#!/bin/sh + +set -ex + +cd "$(dirname "$0")" + +OUT_DIR=/tmp/tidb_tools_test/sync_diff_inspector/output +rm -rf $OUT_DIR +mkdir -p $OUT_DIR + +mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} < ./data.sql + +# tidb +mysql -uroot -h 127.0.0.1 -P 4000 < ./data.sql + +sed "s/\"127.0.0.1\"#MYSQL_HOST/\"${MYSQL_HOST}\"/g" ./config_base.toml | sed "s/3306#MYSQL_PORT/${MYSQL_PORT}/g" > ./config.toml + +echo "compare tables, check result should be pass" +sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true + +check_contains "check pass!!!" $OUT_DIR/sync_diff.log +rm -rf $OUT_DIR/* + +echo "update data to make it different, and data should not be equal" +mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "create table diff_test.t2 (a int, b int, primary key(a));" +mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t3 (a int, b int, primary key(a));" +sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true +check_contains "check failed" $OUT_DIR/sync_diff.log +rm -rf $OUT_DIR/* From 1e471042306be2363b9d494697f6804ab5853669 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Thu, 29 Dec 2022 21:29:58 +0800 Subject: [PATCH 02/20] print information about the skipped table in summary --- sync_diff_inspector/progress/progress_test.go | 14 ++-- sync_diff_inspector/report/report_test.go | 38 +++++----- sync_diff_inspector/source/mysql_shard.go | 71 +++++++++---------- .../table_skip/config_base.toml | 2 +- tests/sync_diff_inspector/table_skip/data.sql | 4 +- tests/sync_diff_inspector/table_skip/run.sh | 2 + 6 files changed, 66 insertions(+), 65 deletions(-) diff --git a/sync_diff_inspector/progress/progress_test.go b/sync_diff_inspector/progress/progress_test.go index 6992483ad..42ecfcd18 100644 --- a/sync_diff_inspector/progress/progress_test.go +++ b/sync_diff_inspector/progress/progress_test.go @@ -24,12 +24,12 @@ import ( func TestProgress(t *testing.T) { p := NewTableProgressPrinter(4, 0) - p.RegisterTable("1", true, true) + p.RegisterTable("1", true, true, 0) p.StartTable("1", 50, true) - p.RegisterTable("2", true, false) + p.RegisterTable("2", true, false, 0) p.StartTable("2", 2, true) p.Inc("2") - p.RegisterTable("3", false, false) + p.RegisterTable("3", false, false, 0) p.StartTable("3", 1, false) p.Inc("2") p.Inc("3") @@ -55,9 +55,9 @@ func TestProgress(t *testing.T) { func TestTableError(t *testing.T) { p := NewTableProgressPrinter(4, 0) - p.RegisterTable("1", true, true) + p.RegisterTable("1", true, true, 0) p.StartTable("1", 50, true) - p.RegisterTable("2", true, true) + p.RegisterTable("2", true, true, 0) p.StartTable("2", 1, true) p.Inc("2") buffer := new(bytes.Buffer) @@ -80,9 +80,9 @@ func TestTableError(t *testing.T) { func TestAllSuccess(t *testing.T) { Init(2, 0) - RegisterTable("1", false, false) + RegisterTable("1", false, false, 0) StartTable("1", 1, true) - RegisterTable("2", false, false) + RegisterTable("2", false, false, 0) StartTable("2", 1, true) Inc("1") Inc("2") diff --git a/sync_diff_inspector/report/report_test.go b/sync_diff_inspector/report/report_test.go index 9bfc324ab..2b4a2d626 100644 --- a/sync_diff_inspector/report/report_test.go +++ b/sync_diff_inspector/report/report_test.go @@ -104,7 +104,7 @@ func TestReport(t *testing.T) { report.CalculateTotalSize(ctx, db) // Test Table Report - report.SetTableStructCheckResult("test", "tbl", true, false) + report.SetTableStructCheckResult("test", "tbl", true, false, 0) report.SetTableDataCheckResult("test", "tbl", true, 100, 200, 222, 222, &chunk.ChunkID{1, 1, 1, 1, 2}) report.SetTableMeetError("test", "tbl", errors.New("eeee")) @@ -121,16 +121,16 @@ func TestReport(t *testing.T) { require.Equal(t, new_report.getSortedTables(), [][]string{{"`atest`.`atbl`", "0", "0"}, {"`ctest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) require.Equal(t, new_report.getDiffRows(), [][]string{}) - new_report.SetTableStructCheckResult("atest", "atbl", true, false) + new_report.SetTableStructCheckResult("atest", "atbl", true, false, 0) new_report.SetTableDataCheckResult("atest", "atbl", false, 111, 222, 333, 333, &chunk.ChunkID{1, 1, 1, 1, 2}) require.Equal(t, new_report.getSortedTables(), [][]string{{"`ctest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) - require.Equal(t, new_report.getDiffRows(), [][]string{{"`atest`.`atbl`", "true", "+111/-222", "333", "333"}}) + require.Equal(t, new_report.getDiffRows(), [][]string{{"`atest`.`atbl`", "succeed", "true", "+111/-222", "333", "333"}}) - new_report.SetTableStructCheckResult("atest", "atbl", false, false) + new_report.SetTableStructCheckResult("atest", "atbl", false, false, 0) require.Equal(t, new_report.getSortedTables(), [][]string{{"`ctest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) - require.Equal(t, new_report.getDiffRows(), [][]string{{"`atest`.`atbl`", "false", "+111/-222", "333", "333"}}) + require.Equal(t, new_report.getDiffRows(), [][]string{{"`atest`.`atbl`", "succeed", "false", "+111/-222", "333", "333"}}) - new_report.SetTableStructCheckResult("ctest", "atbl", false, true) + new_report.SetTableStructCheckResult("ctest", "atbl", false, true, 0) buf := new(bytes.Buffer) new_report.Print(buf) @@ -245,7 +245,7 @@ func TestPrint(t *testing.T) { var buf *bytes.Buffer // All Pass - report.SetTableStructCheckResult("test", "tbl", true, false) + report.SetTableStructCheckResult("test", "tbl", true, false, 0) report.SetTableDataCheckResult("test", "tbl", true, 0, 0, 22, 22, &chunk.ChunkID{0, 0, 0, 0, 1}) buf = new(bytes.Buffer) report.Print(buf) @@ -254,7 +254,7 @@ func TestPrint(t *testing.T) { // Error report.SetTableMeetError("test", "tbl1", errors.New("123")) - report.SetTableStructCheckResult("test", "tbl1", false, false) + report.SetTableStructCheckResult("test", "tbl1", false, false, 0) buf = new(bytes.Buffer) report.Print(buf) require.Equal(t, buf.String(), "Error in comparison process:\n"+ @@ -319,17 +319,17 @@ func TestGetSnapshot(t *testing.T) { } report.Init(tableDiffs, configsBytes[:2], configsBytes[2]) - report.SetTableStructCheckResult("test", "tbl", true, false) + report.SetTableStructCheckResult("test", "tbl", true, false, 0) report.SetTableDataCheckResult("test", "tbl", false, 100, 100, 200, 300, &chunk.ChunkID{0, 0, 0, 1, 10}) report.SetTableDataCheckResult("test", "tbl", true, 0, 0, 300, 300, &chunk.ChunkID{0, 0, 0, 3, 10}) report.SetTableDataCheckResult("test", "tbl", false, 200, 200, 400, 500, &chunk.ChunkID{0, 0, 0, 3, 10}) - report.SetTableStructCheckResult("atest", "tbl", true, false) + report.SetTableStructCheckResult("atest", "tbl", true, false, 0) report.SetTableDataCheckResult("atest", "tbl", false, 100, 100, 500, 600, &chunk.ChunkID{0, 0, 0, 0, 10}) report.SetTableDataCheckResult("atest", "tbl", true, 0, 0, 600, 600, &chunk.ChunkID{0, 0, 0, 3, 10}) report.SetTableDataCheckResult("atest", "tbl", false, 200, 200, 700, 800, &chunk.ChunkID{0, 0, 0, 3, 10}) - report.SetTableStructCheckResult("xtest", "tbl", true, false) + report.SetTableStructCheckResult("xtest", "tbl", true, false, 0) report.SetTableDataCheckResult("xtest", "tbl", false, 100, 100, 800, 900, &chunk.ChunkID{0, 0, 0, 0, 10}) report.SetTableDataCheckResult("xtest", "tbl", true, 0, 0, 900, 900, &chunk.ChunkID{0, 0, 0, 1, 10}) report.SetTableDataCheckResult("xtest", "tbl", false, 200, 200, 1000, 1100, &chunk.ChunkID{0, 0, 0, 3, 10}) @@ -441,13 +441,13 @@ func TestCommitSummary(t *testing.T) { } report.Init(tableDiffs, configsBytes[:2], configsBytes[2]) - report.SetTableStructCheckResult("test", "tbl", true, false) + report.SetTableStructCheckResult("test", "tbl", true, false, 0) report.SetTableDataCheckResult("test", "tbl", true, 100, 200, 400, 400, &chunk.ChunkID{0, 0, 0, 1, 10}) - report.SetTableStructCheckResult("atest", "tbl", true, false) + report.SetTableStructCheckResult("atest", "tbl", true, false, 0) report.SetTableDataCheckResult("atest", "tbl", false, 100, 200, 500, 600, &chunk.ChunkID{0, 0, 0, 2, 10}) - report.SetTableStructCheckResult("xtest", "tbl", false, false) + report.SetTableStructCheckResult("xtest", "tbl", false, false, 0) report.SetTableDataCheckResult("xtest", "tbl", false, 100, 200, 600, 700, &chunk.ChunkID{0, 0, 0, 3, 10}) err = report.CommitSummary() @@ -480,13 +480,13 @@ func TestCommitSummary(t *testing.T) { "| `ytest`.`tbl` | 0 | 0 |\n"+ "+---------------+---------+-----------+\n\n\n"+ "The following tables contains inconsistent data\n\n"+ - "+---------------+--------------------+----------------+---------+-----------+\n"+ - "| TABLE | STRUCTURE EQUALITY | DATA DIFF ROWS | UPCOUNT | DOWNCOUNT |\n"+ - "+---------------+--------------------+----------------+---------+-----------+\n") + "+---------------+---------+--------------------+----------------+---------+-----------+\n"+ + "| TABLE | RESULT | STRUCTURE EQUALITY | DATA DIFF ROWS | UPCOUNT | DOWNCOUNT |\n"+ + "+---------------+---------+--------------------+----------------+---------+-----------+\n") require.Contains(t, str, - "| `atest`.`tbl` | true | +100/-200 | 500 | 600 |\n") + "| `atest`.`tbl` | succeed | true | +100/-200 | 500 | 600 |\n") require.Contains(t, str, - "| `xtest`.`tbl` | false | +100/-200 | 600 | 700 |\n") + "| `xtest`.`tbl` | succeed | false | +100/-200 | 600 | 700 |\n") file.Close() err = os.Remove(filename) diff --git a/sync_diff_inspector/source/mysql_shard.go b/sync_diff_inspector/source/mysql_shard.go index 428e39973..fda86a237 100644 --- a/sync_diff_inspector/source/mysql_shard.go +++ b/sync_diff_inspector/source/mysql_shard.go @@ -103,45 +103,44 @@ func (s *MySQLSources) GetCountAndCrc32(ctx context.Context, tableRange *splitte return &ChecksumInfo{ Count: 0, } - } else { - matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) - infoCh := make(chan *ChecksumInfo, len(s.sourceTablesMap)) - - for _, ms := range matchSources { - go func(ms *common.TableShardSource) { - count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, table.Info, chunk.Where, chunk.Args) - infoCh <- &ChecksumInfo{ - Checksum: checksum, - Count: count, - Err: err, - } - }(ms) - } - defer close(infoCh) - - var ( - err error - totalCount int64 - totalChecksum int64 - ) - - for range matchSources { - info := <-infoCh - // catch the first error - if err == nil && info.Err != nil { - err = info.Err + } + matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) + infoCh := make(chan *ChecksumInfo, len(s.sourceTablesMap)) + + for _, ms := range matchSources { + go func(ms *common.TableShardSource) { + count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, table.Info, chunk.Where, chunk.Args) + infoCh <- &ChecksumInfo{ + Checksum: checksum, + Count: count, + Err: err, } - totalCount += info.Count - totalChecksum ^= info.Checksum + }(ms) + } + defer close(infoCh) + + var ( + err error + totalCount int64 + totalChecksum int64 + ) + + for range matchSources { + info := <-infoCh + // catch the first error + if err == nil && info.Err != nil { + err = info.Err } + totalCount += info.Count + totalChecksum ^= info.Checksum + } - cost := time.Since(beginTime) - return &ChecksumInfo{ - Checksum: totalChecksum, - Count: totalCount, - Err: err, - Cost: cost, - } + cost := time.Since(beginTime) + return &ChecksumInfo{ + Checksum: totalChecksum, + Count: totalCount, + Err: err, + Cost: cost, } } diff --git a/tests/sync_diff_inspector/table_skip/config_base.toml b/tests/sync_diff_inspector/table_skip/config_base.toml index 7130eddda..e03036f57 100644 --- a/tests/sync_diff_inspector/table_skip/config_base.toml +++ b/tests/sync_diff_inspector/table_skip/config_base.toml @@ -18,7 +18,7 @@ check-struct-only = false host = "127.0.0.1"#MYSQL_HOST port = 3306#MYSQL_PORT user = "root" - password = "123456" + password = "" # remove comment if use tidb's snapshot data # snapshot = "2016-10-08 16:45:26" diff --git a/tests/sync_diff_inspector/table_skip/data.sql b/tests/sync_diff_inspector/table_skip/data.sql index 3645cdb7d..a48703433 100644 --- a/tests/sync_diff_inspector/table_skip/data.sql +++ b/tests/sync_diff_inspector/table_skip/data.sql @@ -2,5 +2,5 @@ drop database if exit diff_test; create database diff_test; create table diff_test.t0 (a int, b int, primary key(a)); create table diff_test.t1 (a int, b int, primary key(a)); -insert into diff_test.t0 values (1,1) -insert into diff_test.t1 values (1,1) \ No newline at end of file +insert into diff_test.t0 values (1,1); +insert into diff_test.t1 values (1,1); \ No newline at end of file diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index 324c5c092..3d68fb488 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -23,7 +23,9 @@ rm -rf $OUT_DIR/* echo "update data to make it different, and data should not be equal" mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "create table diff_test.t2 (a int, b int, primary key(a));" +mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t2 values (1,1);" mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t3 (a int, b int, primary key(a));" +mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t3 values (1,1);" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log rm -rf $OUT_DIR/* From d8c7f6ddaac0bf59aa8152d2a474eddedfe88979 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Fri, 30 Dec 2022 01:39:41 +0800 Subject: [PATCH 03/20] print information about the skipped table in summary --- sync_diff_inspector/report/report_test.go | 26 +++++++++++++++++-- tests/sync_diff_inspector/table_skip/data.sql | 2 +- tests/sync_diff_inspector/table_skip/run.sh | 3 ++- 3 files changed, 27 insertions(+), 4 deletions(-) diff --git a/sync_diff_inspector/report/report_test.go b/sync_diff_inspector/report/report_test.go index 2b4a2d626..46bb21d69 100644 --- a/sync_diff_inspector/report/report_test.go +++ b/sync_diff_inspector/report/report_test.go @@ -390,7 +390,9 @@ func TestCommitSummary(t *testing.T) { createTableSQL3 := "create table `xtest`.`tbl`(`a` int, `b` varchar(10), `c` float, `d` datetime, primary key(`a`, `b`))" tableInfo3, err := dbutil.GetTableInfoBySQL(createTableSQL3, parser.New()) require.NoError(t, err) - + createTableSQL4 := "create table `xtest`.`tb1`(`a` int, `b` varchar(10), `c` float, `d` datetime, primary key(`a`, `b`))" + tableInfo4, err := dbutil.GetTableInfoBySQL(createTableSQL4, parser.New()) + require.NoError(t, err) tableDiffs := []*common.TableDiff{ { Schema: "test", @@ -412,6 +414,16 @@ func TestCommitSummary(t *testing.T) { Table: "tbl", Info: tableInfo3, Collation: "[123]", + }, { + Schema: "xtest", + Table: "tb1", + Info: tableInfo4, + Collation: "[123]", + }, { + Schema: "xtest", + Table: "tb2", + Info: tableInfo4, + Collation: "[123]", }, } configs := []*ReportConfig{ @@ -450,13 +462,19 @@ func TestCommitSummary(t *testing.T) { report.SetTableStructCheckResult("xtest", "tbl", false, false, 0) report.SetTableDataCheckResult("xtest", "tbl", false, 100, 200, 600, 700, &chunk.ChunkID{0, 0, 0, 3, 10}) + report.SetTableStructCheckResult("xtest", "tb1", false, true, 1) + report.SetTableDataCheckResult("xtest", "tb1", false, 0, 200, 0, 200, &chunk.ChunkID{0, 0, 0, 4, 10}) + + report.SetTableStructCheckResult("xtest", "tb2", false, true, -1) + report.SetTableDataCheckResult("xtest", "tb2", false, 100, 0, 100, 0, &chunk.ChunkID{0, 0, 0, 5, 10}) + err = report.CommitSummary() require.NoError(t, err) filename := path.Join(outputDir, "summary.txt") file, err := os.Open(filename) require.NoError(t, err) - p := make([]byte, 1024) + p := make([]byte, 2048) file.Read(p) str := string(p) require.Contains(t, str, "Summary\n\n\n\n"+ @@ -487,6 +505,10 @@ func TestCommitSummary(t *testing.T) { "| `atest`.`tbl` | succeed | true | +100/-200 | 500 | 600 |\n") require.Contains(t, str, "| `xtest`.`tbl` | succeed | false | +100/-200 | 600 | 700 |\n") + require.Contains(t, str, + "| `xtest`.`tb1` | skipped | false | +0/-200 | 0 | 200 |\n") + require.Contains(t, str, + "| `xtest`.`tb2` | skipped | false | +100/-0 | 100 | 0 |\n") file.Close() err = os.Remove(filename) diff --git a/tests/sync_diff_inspector/table_skip/data.sql b/tests/sync_diff_inspector/table_skip/data.sql index a48703433..9e8e3230e 100644 --- a/tests/sync_diff_inspector/table_skip/data.sql +++ b/tests/sync_diff_inspector/table_skip/data.sql @@ -1,4 +1,4 @@ -drop database if exit diff_test; +drop database if exists diff_test; create database diff_test; create table diff_test.t0 (a int, b int, primary key(a)); create table diff_test.t1 (a int, b int, primary key(a)); diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index 3d68fb488..8430c3649 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -21,11 +21,12 @@ sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || check_contains "check pass!!!" $OUT_DIR/sync_diff.log rm -rf $OUT_DIR/* -echo "update data to make it different, and data should not be equal" +echo "make some tables exist only upstream or downstream" mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "create table diff_test.t2 (a int, b int, primary key(a));" mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t2 values (1,1);" mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t3 (a int, b int, primary key(a));" mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t3 values (1,1);" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log +cat $OUT_DIR/summary.txt rm -rf $OUT_DIR/* From 72a2bb604f05447624d4b66121cd831e5c7241f8 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Mon, 2 Jan 2023 23:10:49 +0800 Subject: [PATCH 04/20] add parameter `skip-non-existing-table` --- sync_diff_inspector/config/config.go | 3 +++ sync_diff_inspector/source/mysql_shard.go | 7 +++-- sync_diff_inspector/source/source.go | 20 +++++++++----- sync_diff_inspector/source/source_test.go | 27 ++++++++++++------- sync_diff_inspector/source/tidb.go | 8 +++--- .../table_skip/config_base.toml | 3 +++ 6 files changed, 46 insertions(+), 22 deletions(-) diff --git a/sync_diff_inspector/config/config.go b/sync_diff_inspector/config/config.go index 090b74c76..e3bb74005 100644 --- a/sync_diff_inspector/config/config.go +++ b/sync_diff_inspector/config/config.go @@ -374,6 +374,8 @@ type Config struct { CheckStructOnly bool `toml:"check-struct-only" json:"check-struct-only"` // experimental feature: only check table data without table struct CheckDataOnly bool `toml:"check-data-only" json:"-"` + // skip validation for tables that don't exist upstream or downstream + SkipNonExistingTable bool `toml:"skip-non-existing-table" json:"-"` // DMAddr is dm-master's address, the format should like "http://127.0.0.1:8261" DMAddr string `toml:"dm-addr" json:"dm-addr"` // DMTask string `toml:"dm-task" json:"dm-task"` @@ -411,6 +413,7 @@ func NewConfig() *Config { fs.IntVar(&cfg.CheckThreadCount, "check-thread-count", 4, "how many goroutines are created to check data") fs.BoolVar(&cfg.ExportFixSQL, "export-fix-sql", true, "set true if want to compare rows or set to false will only compare checksum") fs.BoolVar(&cfg.CheckStructOnly, "check-struct-only", false, "ignore check table's data") + fs.BoolVar(&cfg.SkipNonExistingTable, "skip-non-existing-table", false, "skip validation for tables that don't exist upstream or downstream") fs.BoolVar(&cfg.CheckDataOnly, "check-data-only", false, "ignore check table's struct") _ = fs.MarkHidden("check-data-only") diff --git a/sync_diff_inspector/source/mysql_shard.go b/sync_diff_inspector/source/mysql_shard.go index fda86a237..eb60479b5 100644 --- a/sync_diff_inspector/source/mysql_shard.go +++ b/sync_diff_inspector/source/mysql_shard.go @@ -296,7 +296,7 @@ func (ms *MultiSourceRowsIterator) Close() { } } -func NewMySQLSources(ctx context.Context, tableDiffs []*common.TableDiff, ds []*config.DataSource, threadCount int, f tableFilter.Filter) (Source, error) { +func NewMySQLSources(ctx context.Context, tableDiffs []*common.TableDiff, ds []*config.DataSource, threadCount int, f tableFilter.Filter, skipNonExistingTable bool) (Source, error) { sourceTablesMap := make(map[string][]*common.TableShardSource) // we should get the real table name // and real table row query from sourceDB. @@ -369,7 +369,10 @@ func NewMySQLSources(ctx context.Context, tableDiffs []*common.TableDiff, ds []* } - tableDiffs = checkTableMatched(tableDiffs, targetUniqueTableMap, sourceTablesAfterRoute) + tableDiffs, err := checkTableMatched(tableDiffs, targetUniqueTableMap, sourceTablesAfterRoute, skipNonExistingTable) + if err != nil { + return nil, errors.Annotatef(err, "please make sure the filter is correct.") + } mss := &MySQLSources{ tableDiffs: tableDiffs, diff --git a/sync_diff_inspector/source/source.go b/sync_diff_inspector/source/source.go index 7190bcf39..65d35c237 100644 --- a/sync_diff_inspector/source/source.go +++ b/sync_diff_inspector/source/source.go @@ -204,18 +204,18 @@ func NewSources(ctx context.Context, cfg *config.Config) (downstream Source, ups bucketSpliterPool := utils.NewWorkerPool(uint(cfg.CheckThreadCount), "bucketIter") // for mysql_shard, it needs `cfg.CheckThreadCount` + `cfg.SplitThreadCount` at most, because it cannot use bucket. mysqlConnCount := cfg.CheckThreadCount + cfg.SplitThreadCount - upstream, err = buildSourceFromCfg(ctx, tableDiffs, mysqlConnCount, bucketSpliterPool, cfg.Task.TargetCheckTables, cfg.Task.SourceInstances...) + upstream, err = buildSourceFromCfg(ctx, tableDiffs, mysqlConnCount, bucketSpliterPool, cfg.SkipNonExistingTable, cfg.Task.TargetCheckTables, cfg.Task.SourceInstances...) if err != nil { return nil, nil, errors.Annotate(err, "from upstream") } - downstream, err = buildSourceFromCfg(ctx, upstream.GetTables(), mysqlConnCount, bucketSpliterPool, cfg.Task.TargetCheckTables, cfg.Task.TargetInstance) + downstream, err = buildSourceFromCfg(ctx, upstream.GetTables(), mysqlConnCount, bucketSpliterPool, cfg.SkipNonExistingTable, cfg.Task.TargetCheckTables, cfg.Task.TargetInstance) if err != nil { return nil, nil, errors.Annotate(err, "from downstream") } return downstream, upstream, nil } -func buildSourceFromCfg(ctx context.Context, tableDiffs []*common.TableDiff, connCount int, bucketSpliterPool *utils.WorkerPool, f tableFilter.Filter, dbs ...*config.DataSource) (Source, error) { +func buildSourceFromCfg(ctx context.Context, tableDiffs []*common.TableDiff, connCount int, bucketSpliterPool *utils.WorkerPool, skipNonExistingTable bool, f tableFilter.Filter, dbs ...*config.DataSource) (Source, error) { if len(dbs) < 1 { return nil, errors.Errorf("no db config detected") } @@ -226,12 +226,12 @@ func buildSourceFromCfg(ctx context.Context, tableDiffs []*common.TableDiff, con if ok { if len(dbs) == 1 { - return NewTiDBSource(ctx, tableDiffs, dbs[0], bucketSpliterPool, f) + return NewTiDBSource(ctx, tableDiffs, dbs[0], bucketSpliterPool, f, skipNonExistingTable) } else { log.Fatal("Don't support check table in multiple tidb instance, please specify one tidb instance.") } } - return NewMySQLSources(ctx, tableDiffs, dbs, connCount, f) + return NewMySQLSources(ctx, tableDiffs, dbs, connCount, f, skipNonExistingTable) } func getAutoSnapshotPosition(cfg *mysql.Config) (string, string, error) { @@ -375,11 +375,14 @@ type RangeIterator interface { Close() } -func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]struct{}, sourceMap map[string]struct{}) []*common.TableDiff { +func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]struct{}, sourceMap map[string]struct{}, skipNonExistingTable bool) ([]*common.TableDiff, error) { // check target exists but source not found for tableDiff := range targetMap { // target table have all passed in tableFilter if _, ok := sourceMap[tableDiff]; !ok { + if !skipNonExistingTable { + return tableDiffs, errors.Errorf("the source has no table to be compared. target-table is `%s`", tableDiff) + } index := getIndexByUniqueID(tableDiffs, tableDiff) if tableDiffs[index].NeedSkippedTable == 0 { tableDiffs[index].NeedSkippedTable = 1 @@ -391,6 +394,9 @@ func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]stru for tableDiff := range sourceMap { // need check source table have passd in tableFilter here if _, ok := targetMap[tableDiff]; !ok { + if !skipNonExistingTable { + return tableDiffs, errors.Errorf("the target has no table to be compared. source-table is `%s`", tableDiff) + } slice := strings.Split(strings.Replace(tableDiff, "`", "", -1), ".") tableDiffs = append(tableDiffs, &common.TableDiff{ Schema: slice[0], @@ -401,7 +407,7 @@ func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]stru } } log.Info("table match check finished") - return tableDiffs + return tableDiffs, nil } // Get the index of table in tableDiffs by uniqueID:`schema`.`table` diff --git a/sync_diff_inspector/source/source_test.go b/sync_diff_inspector/source/source_test.go index 7ad151530..c6d3302a1 100644 --- a/sync_diff_inspector/source/source_test.go +++ b/sync_diff_inspector/source/source_test.go @@ -144,7 +144,7 @@ func TestTiDBSource(t *testing.T) { f, err := filter.Parse([]string{"source_test.*"}) require.NoError(t, err) - tidb, err := NewTiDBSource(ctx, tableDiffs, &config.DataSource{Conn: conn}, utils.NewWorkerPool(1, "bucketIter"), f) + tidb, err := NewTiDBSource(ctx, tableDiffs, &config.DataSource{Conn: conn}, utils.NewWorkerPool(1, "bucketIter"), f, false) require.NoError(t, err) caseFn := []struct { @@ -304,7 +304,7 @@ func TestFallbackToRandomIfRangeIsSet(t *testing.T) { Range: "id < 10", // This should prevent using BucketIterator } - tidb, err := NewTiDBSource(ctx, []*common.TableDiff{table1}, &config.DataSource{Conn: conn}, utils.NewWorkerPool(1, "bucketIter"), f) + tidb, err := NewTiDBSource(ctx, []*common.TableDiff{table1}, &config.DataSource{Conn: conn}, utils.NewWorkerPool(1, "bucketIter"), f, false) require.NoError(t, err) analyze := tidb.GetTableAnalyzer() @@ -374,7 +374,7 @@ func TestMysqlShardSources(t *testing.T) { f, err := filter.Parse([]string{"source_test.*"}) require.NoError(t, err) - shard, err := NewMySQLSources(ctx, tableDiffs, cs, 4, f) + shard, err := NewMySQLSources(ctx, tableDiffs, cs, 4, f, false) require.NoError(t, err) for i := 0; i < len(dbs); i++ { @@ -501,7 +501,7 @@ func TestMysqlRouter(t *testing.T) { f, err := filter.Parse([]string{"*.*"}) require.NoError(t, err) - mysql, err := NewMySQLSources(ctx, tableDiffs, []*config.DataSource{ds}, 4, f) + mysql, err := NewMySQLSources(ctx, tableDiffs, []*config.DataSource{ds}, 4, f, false) require.NoError(t, err) // random splitter @@ -610,7 +610,7 @@ func TestTiDBRouter(t *testing.T) { f, err := filter.Parse([]string{"*.*"}) require.NoError(t, err) - tidb, err := NewTiDBSource(ctx, tableDiffs, ds, utils.NewWorkerPool(1, "bucketIter"), f) + tidb, err := NewTiDBSource(ctx, tableDiffs, ds, utils.NewWorkerPool(1, "bucketIter"), f, false) require.NoError(t, err) infoRows := sqlmock.NewRows([]string{"Table", "Create Table"}).AddRow("test_t", "CREATE TABLE `source_test`.`test1` (`a` int, `b` varchar(24), `c` float, primary key(`a`, `b`))") mock.ExpectQuery("SHOW CREATE TABLE.*").WillReturnRows(infoRows) @@ -929,13 +929,20 @@ func TestCheckTableMatched(t *testing.T) { tmap["`test`.`t1`"] = struct{}{} tmap["`test`.`t2`"] = struct{}{} - tables := checkTableMatched(tableDiffs, tmap, smap) - require.Equal(t, 0, tables[0].NeedSkippedTable) - require.Equal(t, 0, tables[1].NeedSkippedTable) + tables, err := checkTableMatched(tableDiffs, tmap, smap, false) + require.NoError(t, err) - delete(smap, "`test`.`t2`") smap["`test`.`t3`"] = struct{}{} - tables = checkTableMatched(tableDiffs, tmap, smap) + tables, err = checkTableMatched(tableDiffs, tmap, smap, false) + require.Contains(t, err.Error(), "the target has no table to be compared. source-table is ``test`.`t3``") + + delete(smap, "`test`.`t2`") + tables, err = checkTableMatched(tableDiffs, tmap, smap, false) + require.Contains(t, err.Error(), "the source has no table to be compared. target-table is ``test`.`t2``") + + tables, err = checkTableMatched(tableDiffs, tmap, smap, true) + require.NoError(t, err) + require.Equal(t, 0, tables[0].NeedSkippedTable) require.Equal(t, 1, tables[1].NeedSkippedTable) require.Equal(t, -1, tables[2].NeedSkippedTable) } diff --git a/sync_diff_inspector/source/tidb.go b/sync_diff_inspector/source/tidb.go index 9a67ad8aa..2a403b48e 100644 --- a/sync_diff_inspector/source/tidb.go +++ b/sync_diff_inspector/source/tidb.go @@ -194,7 +194,7 @@ func (s *TiDBSource) GetSnapshot() string { return s.snapshot } -func NewTiDBSource(ctx context.Context, tableDiffs []*common.TableDiff, ds *config.DataSource, bucketSpliterPool *utils.WorkerPool, f tableFilter.Filter) (Source, error) { +func NewTiDBSource(ctx context.Context, tableDiffs []*common.TableDiff, ds *config.DataSource, bucketSpliterPool *utils.WorkerPool, f tableFilter.Filter, skipNonExistingTable bool) (Source, error) { sourceTableMap := make(map[string]*common.TableSource) log.Info("find router for tidb source") // we should get the real table name @@ -253,8 +253,10 @@ func NewTiDBSource(ctx context.Context, tableDiffs []*common.TableDiff, ds *conf } } - tableDiffs = checkTableMatched(tableDiffs, targetUniqueTableMap, sourceTablesAfterRoute) - + tableDiffs, err = checkTableMatched(tableDiffs, targetUniqueTableMap, sourceTablesAfterRoute, skipNonExistingTable) + if err != nil { + return nil, errors.Annotatef(err, "please make sure the filter is correct.") + } ts := &TiDBSource{ tableDiffs: tableDiffs, sourceTableMap: sourceTableMap, diff --git a/tests/sync_diff_inspector/table_skip/config_base.toml b/tests/sync_diff_inspector/table_skip/config_base.toml index e03036f57..1081c860d 100644 --- a/tests/sync_diff_inspector/table_skip/config_base.toml +++ b/tests/sync_diff_inspector/table_skip/config_base.toml @@ -12,6 +12,9 @@ export-fix-sql = true # ignore check table's data check-struct-only = false +# skip validation for tables that don't exist upstream or downstream +skip-non-existing-table = true + ######################### Databases config ######################### [data-sources] [data-sources.mysql1] From 5b586f627f375e868f1aecf3ccff52e723bb9faf Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Tue, 3 Jan 2023 22:01:23 +0800 Subject: [PATCH 05/20] print skipped information in progress --- sync_diff_inspector/diff.go | 17 +++------ sync_diff_inspector/progress/progress.go | 34 +++++++++++++++--- sync_diff_inspector/progress/progress_test.go | 6 ++-- sync_diff_inspector/report/report.go | 36 ++++++++++++++----- sync_diff_inspector/report/report_test.go | 3 +- sync_diff_inspector/source/source.go | 9 ++--- 6 files changed, 72 insertions(+), 33 deletions(-) diff --git a/sync_diff_inspector/diff.go b/sync_diff_inspector/diff.go index d2f30f6a9..058d9ed13 100644 --- a/sync_diff_inspector/diff.go +++ b/sync_diff_inspector/diff.go @@ -109,7 +109,7 @@ func (df *Diff) PrintSummary(ctx context.Context) bool { log.Fatal("failed to commit report", zap.Error(err)) } df.report.Print(os.Stdout) - return df.report.Result == report.Pass + return df.report.Result == report.Pass || df.report.Result == report.Skip } func (df *Diff) Close() { @@ -423,19 +423,12 @@ func (df *Diff) consume(ctx context.Context, rangeInfo *splitter.RangeInfo) bool id := rangeInfo.ChunkRange.Index if rangeInfo.ChunkRange.Type == chunk.Empty { dml.node.State = checkpoints.IgnoreState - // for table only exists upstream - if tableDiff.NeedSkippedTable == -1 { + // for tables that only exist upstream or downstream + if tableDiff.NeedSkippedTable != 0 { upCount, _ := dbutil.GetRowCount(ctx, df.upstream.GetDB(), schema, table, "", nil) - isEqual := false - df.report.SetTableDataCheckResult(schema, table, false, int(upCount), 0, upCount, 0, id) - return isEqual - } - // for table only exists downstream - if tableDiff.NeedSkippedTable == 1 { downCount, _ := dbutil.GetRowCount(ctx, df.downstream.GetDB(), schema, table, "", nil) - isEqual := false - df.report.SetTableDataCheckResult(schema, table, isEqual, 0, int(downCount), 0, downCount, id) - return isEqual + df.report.SetTableDataCheckResult(schema, table, false, int(upCount), int(downCount), upCount, downCount, id) + return false } return true } diff --git a/sync_diff_inspector/progress/progress.go b/sync_diff_inspector/progress/progress.go index f0b689945..feab7da33 100644 --- a/sync_diff_inspector/progress/progress.go +++ b/sync_diff_inspector/progress/progress.go @@ -54,6 +54,8 @@ const ( TABLE_STATE_RESULT_DIFFERENT table_state_t = 0x80 TABLE_STATE_HEAD table_state_t = 0xff TABLE_STATE_RESULT_MASK table_state_t = 0xf0 + TABLE_STATE_NOT_EXSIT_UPSTREAM table_state_t = 0x100 + TABLE_STATE_NOT_EXSIT_DOWNSTREAM table_state_t = 0x200 ) type TableProgress struct { @@ -131,7 +133,13 @@ func (tpp *TableProgressPrinter) RegisterTable(name string, isFailed bool, isDon var state table_state_t if isFailed { if isDone { - state = TABLE_STATE_RESULT_FAIL_STRUCTURE_DONE | TABLE_STATE_REGISTER + if isAllExist == 1 { + state = TABLE_STATE_NOT_EXSIT_UPSTREAM | TABLE_STATE_REGISTER + } else if isAllExist == -1 { + state = TABLE_STATE_NOT_EXSIT_DOWNSTREAM | TABLE_STATE_REGISTER + } else { + state = TABLE_STATE_RESULT_FAIL_STRUCTURE_DONE | TABLE_STATE_REGISTER + } } else { state = TABLE_STATE_RESULT_FAIL_STRUCTURE_CONTINUE | TABLE_STATE_REGISTER } @@ -181,6 +189,7 @@ func (tpp *TableProgressPrinter) PrintSummary() { tpp.tableNums, ) } else { + SkippedNum := 0 for p := tpp.tableFailList.Front(); p != nil; p = p.Next() { tp := p.Value.(*TableProgress) if tp.state&(TABLE_STATE_RESULT_FAIL_STRUCTURE_DONE|TABLE_STATE_RESULT_FAIL_STRUCTURE_CONTINUE) != 0 { @@ -189,10 +198,18 @@ func (tpp *TableProgressPrinter) PrintSummary() { if tp.state&(TABLE_STATE_RESULT_DIFFERENT) != 0 { fixStr = fmt.Sprintf("%sThe data of `%s` is not equal.\n", fixStr, tp.name) } + if tp.state&(TABLE_STATE_NOT_EXSIT_DOWNSTREAM) != 0 { + fixStr = fmt.Sprintf("%sThe data of `%s` does not exist in downstream database.\n", fixStr, tp.name) + SkippedNum++ + } + if tp.state&(TABLE_STATE_NOT_EXSIT_UPSTREAM) != 0 { + fixStr = fmt.Sprintf("%sThe data of `%s` does not exist in upstream database.\n", fixStr, tp.name) + SkippedNum++ + } } fixStr = fmt.Sprintf( - "%s\nThe rest of the tables are all equal.\nThe patch file has been generated to './output_dir/patch.sql'\nYou can view the comparison details through './output_dir/sync_diff_inspector.log'\n", - fixStr, + "%s\nThe rest of the tables are all equal.\nA total of %d tables have been compared, %d tables finished, %d tables failed, %d tables skipped.\nThe patch file has been generated to './output_dir/patch.sql'\nYou can view the comparison details through './output_dir/sync_diff_inspector.log'\n", + fixStr, tpp.tableNums, tpp.tableNums-tpp.tableFailList.Len(), tpp.tableFailList.Len()-SkippedNum, SkippedNum, ) } @@ -330,13 +347,22 @@ func (tpp *TableProgressPrinter) flush(stateIsChanged bool) { // 5. structure is different and data is different switch tp.state & 0xf { case TABLE_STATE_PRESTART: - switch tp.state & TABLE_STATE_RESULT_MASK { + switch tp.state & 0xff0 { case TABLE_STATE_RESULT_OK: fixStr = fmt.Sprintf("%sComparing the table structure of `%s` ... equivalent\n", fixStr, tp.name) dynStr = fmt.Sprintf("%sComparing the table data of `%s` ...\n", dynStr, tp.name) tpp.lines++ tpp.progressTableNums++ tp.state = TABLE_STATE_COMPARING + case TABLE_STATE_NOT_EXSIT_UPSTREAM: + fallthrough + case TABLE_STATE_NOT_EXSIT_DOWNSTREAM: + dynStr = fmt.Sprintf("%sComparing the table data of `%s` ...skipped\n", dynStr, tp.name) + tpp.tableFailList.PushBack(tp) + preNode := p.Prev() + tpp.tableList.Remove(p) + p = preNode + tpp.finishTableNums++ case TABLE_STATE_RESULT_FAIL_STRUCTURE_DONE: fixStr = fmt.Sprintf("%sComparing the table structure of `%s` ... failure\n", fixStr, tp.name) tpp.tableFailList.PushBack(tp) diff --git a/sync_diff_inspector/progress/progress_test.go b/sync_diff_inspector/progress/progress_test.go index 42ecfcd18..2bb416dcd 100644 --- a/sync_diff_inspector/progress/progress_test.go +++ b/sync_diff_inspector/progress/progress_test.go @@ -24,7 +24,7 @@ import ( func TestProgress(t *testing.T) { p := NewTableProgressPrinter(4, 0) - p.RegisterTable("1", true, true, 0) + p.RegisterTable("1", true, true, 1) p.StartTable("1", 50, true) p.RegisterTable("2", true, false, 0) p.StartTable("2", 2, true) @@ -47,8 +47,8 @@ func TestProgress(t *testing.T) { require.Equal( t, buffer.String(), - "\x1b[1A\x1b[J\nSummary:\n\nThe structure of `1` is not equal.\nThe structure of `2` is not equal.\nThe data of `4` is not equal.\n"+ - "\nThe rest of the tables are all equal.\nThe patch file has been generated to './output_dir/patch.sql'\n"+ + "\x1b[1A\x1b[J\nSummary:\n\nThe data of `1` does not exist in upstream database.\nThe structure of `2` is not equal.\nThe data of `4` is not equal.\n"+ + "\nThe rest of the tables are all equal.\nA total of 4 tables have been compared, 1 tables finished, 2 tables failed, 1 tables skipped.\nThe patch file has been generated to './output_dir/patch.sql'\n"+ "You can view the comparison details through './output_dir/sync_diff_inspector.log'\n\n", ) } diff --git a/sync_diff_inspector/report/report.go b/sync_diff_inspector/report/report.go index cb1f5e0a9..d5cec6f0c 100644 --- a/sync_diff_inspector/report/report.go +++ b/sync_diff_inspector/report/report.go @@ -42,6 +42,7 @@ const ( Pass = "pass" // Fail means not all data or struct of tables are equal Fail = "fail" + Skip = "skip" Error = "error" ) @@ -80,6 +81,7 @@ type Report struct { Result string `json:"-"` // Result is pass or fail PassNum int32 `json:"-"` // The pass number of tables FailedNum int32 `json:"-"` // The failed number of tables + SkippedNum int32 `json:"-"` // Theskippeded number of tables TableResults map[string]map[string]*TableResult `json:"table-results"` // TableResult saved the map of `schema` => `table` => `tableResult` StartTime time.Time `json:"start-time"` Duration time.Duration `json:"time-duration"` @@ -159,8 +161,7 @@ func (r *Report) CalculateTotalSize(ctx context.Context, db *sql.DB) { for schema, tableMap := range r.TableResults { for table := range tableMap { size, err := utils.GetTableSize(ctx, db, schema, table) - tableSkipped := r.TableResults[schema][table].TableSkipped - if (size == 0 || err != nil) && tableSkipped == 0 { + if size == 0 || err != nil { log.Warn("fail to get the correct size of table, if you want to get the correct size, please analyze the corresponding tables", zap.String("table", dbutil.TableName(schema, table)), zap.Error(err)) } else { r.TotalSize += size @@ -171,11 +172,13 @@ func (r *Report) CalculateTotalSize(ctx context.Context, db *sql.DB) { // CommitSummary commit summary info func (r *Report) CommitSummary() error { - passNum, failedNum := int32(0), int32(0) + passNum, failedNum, skippedNum := int32(0), int32(0), int32(0) for _, tableMap := range r.TableResults { for _, result := range tableMap { if result.StructEqual && result.DataEqual { passNum++ + } else if result.TableSkipped != 0 { + skippedNum++ } else { failedNum++ } @@ -183,6 +186,7 @@ func (r *Report) CommitSummary() error { } r.PassNum = passNum r.FailedNum = failedNum + r.SkippedNum = skippedNum summaryPath := filepath.Join(r.task.OutputDir, "summary.txt") summaryFile, err := os.Create(summaryPath) if err != nil { @@ -213,11 +217,11 @@ func (r *Report) CommitSummary() error { summaryFile.WriteString(tableString.String()) summaryFile.WriteString("\n\n") } - if r.Result == Fail { + if r.Result == Fail || r.Result == Skip { summaryFile.WriteString("The following tables contains inconsistent data\n\n") tableString := &strings.Builder{} table := tablewriter.NewWriter(tableString) - table.SetHeader([]string{"Table", "RESULT", "Structure equality", "Data diff rows", "UpCount", "DownCount"}) + table.SetHeader([]string{"Table", "Result", "Structure equality", "Data diff rows", "UpCount", "DownCount"}) diffRows := r.getDiffRows() for _, v := range diffRows { table.Append(v) @@ -234,25 +238,33 @@ func (r *Report) CommitSummary() error { func (r *Report) Print(w io.Writer) error { var summary strings.Builder if r.Result == Pass { - summary.WriteString(fmt.Sprintf("A total of %d table have been compared and all are equal.\n", r.FailedNum+r.PassNum)) + summary.WriteString(fmt.Sprintf("A total of %d table have been compared and all are equal.\n", r.FailedNum+r.PassNum+r.SkippedNum)) summary.WriteString(fmt.Sprintf("You can view the comparision details through '%s/%s'\n", r.task.OutputDir, config.LogFileName)) - } else if r.Result == Fail { + } else if r.Result == Skip || r.Result == Fail { for schema, tableMap := range r.TableResults { for table, result := range tableMap { if !result.StructEqual { if result.DataSkip { - summary.WriteString(fmt.Sprintf("The structure of %s is not equal, and data-check is skipped\n", dbutil.TableName(schema, table))) + if result.TableSkipped == 1 { + summary.WriteString(fmt.Sprintf("The data of %s does not exist in upstream database\n", dbutil.TableName(schema, table))) + } else if result.TableSkipped == -1 { + summary.WriteString(fmt.Sprintf("The data of %s does not exist in downstream database\n", dbutil.TableName(schema, table))) + } else { + summary.WriteString(fmt.Sprintf("The structure of %s is not equal, and data-check is skipped\n", dbutil.TableName(schema, table))) + } } else { summary.WriteString(fmt.Sprintf("The structure of %s is not equal\n", dbutil.TableName(schema, table))) } } - if !result.DataEqual { + if !result.DataEqual && result.TableSkipped == 0 { summary.WriteString(fmt.Sprintf("The data of %s is not equal\n", dbutil.TableName(schema, table))) } } } summary.WriteString("\n") summary.WriteString("The rest of tables are all equal.\n") + summary.WriteString("\n") + summary.WriteString(fmt.Sprintf("A total of %d tables have been compared, %d tables finished, %d tables failed, %d tables skipped.\n", r.FailedNum+r.PassNum+r.SkippedNum, r.PassNum, r.FailedNum, r.SkippedNum)) summary.WriteString(fmt.Sprintf("The patch file has been generated in \n\t'%s/'\n", r.task.FixDir)) summary.WriteString(fmt.Sprintf("You can view the comparision details through '%s/%s'\n", r.task.OutputDir, config.LogFileName)) } else { @@ -310,6 +322,9 @@ func (r *Report) SetTableStructCheckResult(schema, table string, equal bool, ski if !equal && r.Result != Error { r.Result = Fail } + if exist != 0 && r.Result != Error { + r.Result = Skip + } } // SetTableDataCheckResult sets the data check result for table. @@ -336,6 +351,9 @@ func (r *Report) SetTableDataCheckResult(schema, table string, equal bool, rowsA if !equal && r.Result != Error { r.Result = Fail } + if result.TableSkipped != 0 && r.Result != Error { + r.Result = Skip + } } // SetTableMeetError sets meet error when check the table. diff --git a/sync_diff_inspector/report/report_test.go b/sync_diff_inspector/report/report_test.go index 46bb21d69..ac9398843 100644 --- a/sync_diff_inspector/report/report_test.go +++ b/sync_diff_inspector/report/report_test.go @@ -139,7 +139,8 @@ func TestReport(t *testing.T) { require.Contains(t, info, "The data of `atest`.`atbl` is not equal\n") require.Contains(t, info, "The structure of `ctest`.`atbl` is not equal, and data-check is skipped\n") require.Contains(t, info, "\n"+ - "The rest of tables are all equal.\n"+ + "The rest of tables are all equal.\n\n"+ + "A total of 0 tables have been compared, 0 tables finished, 0 tables failed, 0 tables skipped.\n"+ "The patch file has been generated in \n\t'output_dir/123456/fix-on-tidb1/'\n"+ "You can view the comparision details through 'output_dir/sync_diff.log'\n") } diff --git a/sync_diff_inspector/source/source.go b/sync_diff_inspector/source/source.go index 65d35c237..131f03bba 100644 --- a/sync_diff_inspector/source/source.go +++ b/sync_diff_inspector/source/source.go @@ -383,7 +383,7 @@ func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]stru if !skipNonExistingTable { return tableDiffs, errors.Errorf("the source has no table to be compared. target-table is `%s`", tableDiff) } - index := getIndexByUniqueID(tableDiffs, tableDiff) + index := getIndexMapForTable(tableDiffs, tableDiff)[tableDiff] if tableDiffs[index].NeedSkippedTable == 0 { tableDiffs[index].NeedSkippedTable = 1 log.Info("the source has no table to be compared", zap.String("target-table", tableDiff)) @@ -411,12 +411,13 @@ func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]stru } // Get the index of table in tableDiffs by uniqueID:`schema`.`table` -func getIndexByUniqueID(tableDiffs []*common.TableDiff, uniqueID string) int { +func getIndexMapForTable(tableDiffs []*common.TableDiff, uniqueID string) map[string]int { + tableIndexMap := make(map[string]int) for i := 0; i < len(tableDiffs); i++ { tableUniqueID := utils.UniqueID(tableDiffs[i].Schema, tableDiffs[i].Table) if tableUniqueID == uniqueID { - return i + tableIndexMap[uniqueID] = i } } - return 0 + return tableIndexMap } From 54dc8df6364a6a091367bff67e16f6de1ff9e434 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Wed, 4 Jan 2023 20:25:03 +0800 Subject: [PATCH 06/20] fix some error --- sync_diff_inspector/diff.go | 14 +++--- sync_diff_inspector/progress/progress.go | 20 ++++---- sync_diff_inspector/report/report.go | 50 ++++++++++++------- sync_diff_inspector/source/chunks_iter.go | 2 +- .../source/common/table_diff.go | 6 +++ sync_diff_inspector/source/mysql_shard.go | 6 +-- sync_diff_inspector/source/source.go | 11 +++- tests/sync_diff_inspector/table_skip/run.sh | 17 ++++++- 8 files changed, 84 insertions(+), 42 deletions(-) diff --git a/sync_diff_inspector/diff.go b/sync_diff_inspector/diff.go index 058d9ed13..fb43ba20e 100644 --- a/sync_diff_inspector/diff.go +++ b/sync_diff_inspector/diff.go @@ -109,7 +109,7 @@ func (df *Diff) PrintSummary(ctx context.Context) bool { log.Fatal("failed to commit report", zap.Error(err)) } df.report.Print(os.Stdout) - return df.report.Result == report.Pass || df.report.Result == report.Skip + return df.report.Result == report.Pass } func (df *Diff) Close() { @@ -298,8 +298,8 @@ func (df *Diff) StructEqual(ctx context.Context) error { } for ; tableIndex < len(tables); tableIndex++ { var isEqual, isSkip bool - isAllExist := tables[tableIndex].NeedSkippedTable - if isAllExist == 0 { + isAllTableExist := tables[tableIndex].NeedSkippedTable + if source.AllTableExist(tables[tableIndex]) { var err error isEqual, isSkip, err = df.compareStruct(ctx, tableIndex) if err != nil { @@ -308,8 +308,8 @@ func (df *Diff) StructEqual(ctx context.Context) error { } else { isEqual, isSkip = false, true } - progress.RegisterTable(dbutil.TableName(tables[tableIndex].Schema, tables[tableIndex].Table), !isEqual, isSkip, isAllExist) - df.report.SetTableStructCheckResult(tables[tableIndex].Schema, tables[tableIndex].Table, isEqual, isSkip, isAllExist) + progress.RegisterTable(dbutil.TableName(tables[tableIndex].Schema, tables[tableIndex].Table), !isEqual, isSkip, isAllTableExist) + df.report.SetTableStructCheckResult(tables[tableIndex].Schema, tables[tableIndex].Table, isEqual, isSkip, isAllTableExist) } return nil } @@ -423,8 +423,8 @@ func (df *Diff) consume(ctx context.Context, rangeInfo *splitter.RangeInfo) bool id := rangeInfo.ChunkRange.Index if rangeInfo.ChunkRange.Type == chunk.Empty { dml.node.State = checkpoints.IgnoreState - // for tables that only exist upstream or downstream - if tableDiff.NeedSkippedTable != 0 { + // for tables that don't exist upstream or downstream + if !source.AllTableExist(tableDiff) { upCount, _ := dbutil.GetRowCount(ctx, df.upstream.GetDB(), schema, table, "", nil) downCount, _ := dbutil.GetRowCount(ctx, df.downstream.GetDB(), schema, table, "", nil) df.report.SetTableDataCheckResult(schema, table, false, int(upCount), int(downCount), upCount, downCount, id) diff --git a/sync_diff_inspector/progress/progress.go b/sync_diff_inspector/progress/progress.go index feab7da33..94be4fbc5 100644 --- a/sync_diff_inspector/progress/progress.go +++ b/sync_diff_inspector/progress/progress.go @@ -20,6 +20,8 @@ import ( "os" "strings" "time" + + "github.com/pingcap/tidb-tools/sync_diff_inspector/source/common" ) type TableProgressPrinter struct { @@ -53,7 +55,7 @@ const ( TABLE_STATE_RESULT_FAIL_STRUCTURE_PASS table_state_t = 0x40 TABLE_STATE_RESULT_DIFFERENT table_state_t = 0x80 TABLE_STATE_HEAD table_state_t = 0xff - TABLE_STATE_RESULT_MASK table_state_t = 0xf0 + TABLE_STATE_RESULT_MASK table_state_t = 0xff0 TABLE_STATE_NOT_EXSIT_UPSTREAM table_state_t = 0x100 TABLE_STATE_NOT_EXSIT_DOWNSTREAM table_state_t = 0x200 ) @@ -129,13 +131,13 @@ func (tpp *TableProgressPrinter) UpdateTotal(name string, total int, stopUpdate } } -func (tpp *TableProgressPrinter) RegisterTable(name string, isFailed bool, isDone bool, isAllExist int) { +func (tpp *TableProgressPrinter) RegisterTable(name string, isFailed bool, isDone bool, isExist int) { var state table_state_t if isFailed { if isDone { - if isAllExist == 1 { + if isExist == common.DownstreamTableLackFlag { state = TABLE_STATE_NOT_EXSIT_UPSTREAM | TABLE_STATE_REGISTER - } else if isAllExist == -1 { + } else if isExist == common.UpstreamTableLackFlag { state = TABLE_STATE_NOT_EXSIT_DOWNSTREAM | TABLE_STATE_REGISTER } else { state = TABLE_STATE_RESULT_FAIL_STRUCTURE_DONE | TABLE_STATE_REGISTER @@ -347,16 +349,14 @@ func (tpp *TableProgressPrinter) flush(stateIsChanged bool) { // 5. structure is different and data is different switch tp.state & 0xf { case TABLE_STATE_PRESTART: - switch tp.state & 0xff0 { + switch tp.state & TABLE_STATE_RESULT_MASK { case TABLE_STATE_RESULT_OK: fixStr = fmt.Sprintf("%sComparing the table structure of `%s` ... equivalent\n", fixStr, tp.name) dynStr = fmt.Sprintf("%sComparing the table data of `%s` ...\n", dynStr, tp.name) tpp.lines++ tpp.progressTableNums++ tp.state = TABLE_STATE_COMPARING - case TABLE_STATE_NOT_EXSIT_UPSTREAM: - fallthrough - case TABLE_STATE_NOT_EXSIT_DOWNSTREAM: + case TABLE_STATE_NOT_EXSIT_UPSTREAM, TABLE_STATE_NOT_EXSIT_DOWNSTREAM: dynStr = fmt.Sprintf("%sComparing the table data of `%s` ...skipped\n", dynStr, tp.name) tpp.tableFailList.PushBack(tp) preNode := p.Prev() @@ -436,9 +436,9 @@ func UpdateTotal(name string, total int, stopUpdate bool) { } } -func RegisterTable(name string, isFailed bool, isDone bool, isAllExist int) { +func RegisterTable(name string, isFailed bool, isDone bool, isExist int) { if progress_ != nil { - progress_.RegisterTable(name, isFailed, isDone, isAllExist) + progress_.RegisterTable(name, isFailed, isDone, isExist) } } diff --git a/sync_diff_inspector/report/report.go b/sync_diff_inspector/report/report.go index d5cec6f0c..9d33662ff 100644 --- a/sync_diff_inspector/report/report.go +++ b/sync_diff_inspector/report/report.go @@ -42,7 +42,6 @@ const ( Pass = "pass" // Fail means not all data or struct of tables are equal Fail = "fail" - Skip = "skip" Error = "error" ) @@ -133,7 +132,7 @@ func (r *Report) getDiffRows() [][]string { } diffRow := make([]string, 0) diffRow = append(diffRow, dbutil.TableName(schema, table)) - if result.TableSkipped != 0 { + if !AllTableExist(result) { diffRow = append(diffRow, "skipped") } else { diffRow = append(diffRow, "succeed") @@ -177,7 +176,7 @@ func (r *Report) CommitSummary() error { for _, result := range tableMap { if result.StructEqual && result.DataEqual { passNum++ - } else if result.TableSkipped != 0 { + } else if !AllTableExist(result) { skippedNum++ } else { failedNum++ @@ -217,7 +216,7 @@ func (r *Report) CommitSummary() error { summaryFile.WriteString(tableString.String()) summaryFile.WriteString("\n\n") } - if r.Result == Fail || r.Result == Skip { + if r.Result == Fail || r.SkippedNum != 0 { summaryFile.WriteString("The following tables contains inconsistent data\n\n") tableString := &strings.Builder{} table := tablewriter.NewWriter(tableString) @@ -237,17 +236,17 @@ func (r *Report) CommitSummary() error { func (r *Report) Print(w io.Writer) error { var summary strings.Builder - if r.Result == Pass { + if r.Result == Pass && r.SkippedNum == 0 { summary.WriteString(fmt.Sprintf("A total of %d table have been compared and all are equal.\n", r.FailedNum+r.PassNum+r.SkippedNum)) summary.WriteString(fmt.Sprintf("You can view the comparision details through '%s/%s'\n", r.task.OutputDir, config.LogFileName)) - } else if r.Result == Skip || r.Result == Fail { + } else if r.Result == Fail || r.SkippedNum != 0 { for schema, tableMap := range r.TableResults { for table, result := range tableMap { if !result.StructEqual { if result.DataSkip { - if result.TableSkipped == 1 { + if UpstreamTableLack(result) { summary.WriteString(fmt.Sprintf("The data of %s does not exist in upstream database\n", dbutil.TableName(schema, table))) - } else if result.TableSkipped == -1 { + } else if DownstreamTableLack(result) { summary.WriteString(fmt.Sprintf("The data of %s does not exist in downstream database\n", dbutil.TableName(schema, table))) } else { summary.WriteString(fmt.Sprintf("The structure of %s is not equal, and data-check is skipped\n", dbutil.TableName(schema, table))) @@ -256,7 +255,7 @@ func (r *Report) Print(w io.Writer) error { summary.WriteString(fmt.Sprintf("The structure of %s is not equal\n", dbutil.TableName(schema, table))) } } - if !result.DataEqual && result.TableSkipped == 0 { + if !result.DataEqual && AllTableExist(result) { summary.WriteString(fmt.Sprintf("The data of %s is not equal\n", dbutil.TableName(schema, table))) } } @@ -319,12 +318,9 @@ func (r *Report) SetTableStructCheckResult(schema, table string, equal bool, ski tableResult.StructEqual = equal tableResult.DataSkip = skip tableResult.TableSkipped = exist - if !equal && r.Result != Error { + if !equal && AllTableExist(tableResult) && r.Result != Error { r.Result = Fail } - if exist != 0 && r.Result != Error { - r.Result = Skip - } } // SetTableDataCheckResult sets the data check result for table. @@ -344,16 +340,13 @@ func (r *Report) SetTableDataCheckResult(schema, table string, equal bool, rowsA } result.ChunkMap[id.ToString()].RowsAdd += rowsAdd result.ChunkMap[id.ToString()].RowsDelete += rowsDelete - if r.Result != Error { + if r.Result != Error && AllTableExist(result) { r.Result = Fail } } - if !equal && r.Result != Error { + if !equal && AllTableExist(result) && r.Result != Error { r.Result = Fail } - if result.TableSkipped != 0 && r.Result != Error { - r.Result = Skip - } } // SetTableMeetError sets meet error when check the table. @@ -423,3 +416,24 @@ func (r *Report) GetSnapshot(chunkID *chunk.ChunkID, schema, table string) (*Rep task: task, }, nil } + +func AllTableExist(result *TableResult) bool { + if result.TableSkipped == common.AllTableExistFlag { + return true + } + return false +} + +func UpstreamTableLack(result *TableResult) bool { + if result.TableSkipped == common.UpstreamTableLackFlag { + return true + } + return false +} + +func DownstreamTableLack(result *TableResult) bool { + if result.TableSkipped == common.DownstreamTableLackFlag { + return true + } + return false +} diff --git a/sync_diff_inspector/source/chunks_iter.go b/sync_diff_inspector/source/chunks_iter.go index 104ff3435..0268b7ad3 100644 --- a/sync_diff_inspector/source/chunks_iter.go +++ b/sync_diff_inspector/source/chunks_iter.go @@ -103,7 +103,7 @@ func (t *ChunksIterator) produceChunks(ctx context.Context, startRange *splitter for ; t.nextTableIndex < len(t.TableDiffs); t.nextTableIndex++ { curTableIndex := t.nextTableIndex // skip data-check, but still need to send a empty chunk to make checkpoint continuous - if t.TableDiffs[curTableIndex].IgnoreDataCheck || t.TableDiffs[curTableIndex].NeedSkippedTable != 0 { + if t.TableDiffs[curTableIndex].IgnoreDataCheck || !AllTableExist(t.TableDiffs[curTableIndex]) { pool.Apply(func() { table := t.TableDiffs[curTableIndex] progressID := dbutil.TableName(table.Schema, table.Table) diff --git a/sync_diff_inspector/source/common/table_diff.go b/sync_diff_inspector/source/common/table_diff.go index 33816a38f..e4e61ed80 100644 --- a/sync_diff_inspector/source/common/table_diff.go +++ b/sync_diff_inspector/source/common/table_diff.go @@ -71,3 +71,9 @@ type TableDiff struct { // NeedSkippedTable = 0: the table exists both upstream and downstream. NeedSkippedTable int `json:"-"` } + +const ( + AllTableExistFlag = 0 + UpstreamTableLackFlag = -1 + DownstreamTableLackFlag = 1 +) diff --git a/sync_diff_inspector/source/mysql_shard.go b/sync_diff_inspector/source/mysql_shard.go index eb60479b5..ac1477516 100644 --- a/sync_diff_inspector/source/mysql_shard.go +++ b/sync_diff_inspector/source/mysql_shard.go @@ -99,7 +99,7 @@ func (s *MySQLSources) GetCountAndCrc32(ctx context.Context, tableRange *splitte chunk := tableRange.GetChunk() // for tables that do not exist upstream or downstream - if table.NeedSkippedTable != 0 { + if !AllTableExist(table) { return &ChecksumInfo{ Count: 0, } @@ -169,7 +169,7 @@ func (s *MySQLSources) GetRowsIterator(ctx context.Context, tableRange *splitter table := s.tableDiffs[tableRange.GetTableIndex()] // for tables that do not exist upstream or downstream - if table.NeedSkippedTable != 0 { + if !AllTableExist(table) { return nil, nil } matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) @@ -234,7 +234,7 @@ func (s *MySQLSources) GetSnapshot() string { func (s *MySQLSources) GetSourceStructInfo(ctx context.Context, tableIndex int) ([]*model.TableInfo, error) { tableDiff := s.GetTables()[tableIndex] // for tables that do not exist upstream or downstream - if tableDiff.NeedSkippedTable != 0 { + if !AllTableExist(tableDiff) { return nil, nil } tableSources := getMatchedSourcesForTable(s.sourceTablesMap, tableDiff) diff --git a/sync_diff_inspector/source/source.go b/sync_diff_inspector/source/source.go index 131f03bba..fa13d9e43 100644 --- a/sync_diff_inspector/source/source.go +++ b/sync_diff_inspector/source/source.go @@ -385,7 +385,7 @@ func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]stru } index := getIndexMapForTable(tableDiffs, tableDiff)[tableDiff] if tableDiffs[index].NeedSkippedTable == 0 { - tableDiffs[index].NeedSkippedTable = 1 + tableDiffs[index].NeedSkippedTable = common.UpstreamTableLackFlag log.Info("the source has no table to be compared", zap.String("target-table", tableDiff)) } } @@ -401,7 +401,7 @@ func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]stru tableDiffs = append(tableDiffs, &common.TableDiff{ Schema: slice[0], Table: slice[1], - NeedSkippedTable: -1, + NeedSkippedTable: common.DownstreamTableLackFlag, }) log.Info("the target has no table to be compared", zap.String("source-table", tableDiff)) } @@ -421,3 +421,10 @@ func getIndexMapForTable(tableDiffs []*common.TableDiff, uniqueID string) map[st } return tableIndexMap } + +func AllTableExist(tableDiffs *common.TableDiff) bool { + if tableDiffs.NeedSkippedTable == common.AllTableExistFlag { + return true + } + return false +} diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index 8430c3649..d63018aba 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -17,7 +17,6 @@ sed "s/\"127.0.0.1\"#MYSQL_HOST/\"${MYSQL_HOST}\"/g" ./config_base.toml | sed "s echo "compare tables, check result should be pass" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true - check_contains "check pass!!!" $OUT_DIR/sync_diff.log rm -rf $OUT_DIR/* @@ -27,6 +26,22 @@ mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t2 valu mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t3 (a int, b int, primary key(a));" mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t3 values (1,1);" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true +check_contains "check pass" $OUT_DIR/sync_diff.log +rm -rf $OUT_DIR/* + +echo "make some table data not equal" +mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t1 values (2,2);" +sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true +check_contains "check failed" $OUT_DIR/sync_diff.log +rm -rf $OUT_DIR/* + +echo "make some table structure not equal" +mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "create table diff_test.t4 (a int, b int, c int,primary key(a));" +mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t4 values (1,1,1);" +mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t4 (a int, b int, primary key(a));" +sync_diff_inspector --config=./config.toml #> $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log cat $OUT_DIR/summary.txt rm -rf $OUT_DIR/* + +echo "table_config test passed" \ No newline at end of file From f701426f968c158b352b9ca7e8b8514f1c5e34c5 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Wed, 4 Jan 2023 21:06:35 +0800 Subject: [PATCH 07/20] fix some error --- sync_diff_inspector/source/common/table_diff.go | 4 ++-- tests/sync_diff_inspector/table_skip/run.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sync_diff_inspector/source/common/table_diff.go b/sync_diff_inspector/source/common/table_diff.go index e4e61ed80..1320a3cde 100644 --- a/sync_diff_inspector/source/common/table_diff.go +++ b/sync_diff_inspector/source/common/table_diff.go @@ -74,6 +74,6 @@ type TableDiff struct { const ( AllTableExistFlag = 0 - UpstreamTableLackFlag = -1 - DownstreamTableLackFlag = 1 + DownstreamTableLackFlag = -1 + UpstreamTableLackFlag = 1 ) diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index d63018aba..c960cee82 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -39,9 +39,9 @@ echo "make some table structure not equal" mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "create table diff_test.t4 (a int, b int, c int,primary key(a));" mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t4 values (1,1,1);" mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t4 (a int, b int, primary key(a));" -sync_diff_inspector --config=./config.toml #> $OUT_DIR/table_skip_diff.output || true +sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log cat $OUT_DIR/summary.txt rm -rf $OUT_DIR/* -echo "table_config test passed" \ No newline at end of file +echo "table_skip test passed" \ No newline at end of file From 5a4234247005a2029c2dc499a196744671cb4221 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Wed, 4 Jan 2023 22:54:10 +0800 Subject: [PATCH 08/20] update test --- sync_diff_inspector/progress/progress.go | 4 ++-- sync_diff_inspector/progress/progress_test.go | 6 ++++++ sync_diff_inspector/report/report_test.go | 15 ++++++++++++--- tests/sync_diff_inspector/table_skip/run.sh | 1 + 4 files changed, 21 insertions(+), 5 deletions(-) diff --git a/sync_diff_inspector/progress/progress.go b/sync_diff_inspector/progress/progress.go index 94be4fbc5..827b6c8f7 100644 --- a/sync_diff_inspector/progress/progress.go +++ b/sync_diff_inspector/progress/progress.go @@ -135,9 +135,9 @@ func (tpp *TableProgressPrinter) RegisterTable(name string, isFailed bool, isDon var state table_state_t if isFailed { if isDone { - if isExist == common.DownstreamTableLackFlag { + if isExist == common.UpstreamTableLackFlag { state = TABLE_STATE_NOT_EXSIT_UPSTREAM | TABLE_STATE_REGISTER - } else if isExist == common.UpstreamTableLackFlag { + } else if isExist == common.DownstreamTableLackFlag { state = TABLE_STATE_NOT_EXSIT_DOWNSTREAM | TABLE_STATE_REGISTER } else { state = TABLE_STATE_RESULT_FAIL_STRUCTURE_DONE | TABLE_STATE_REGISTER diff --git a/sync_diff_inspector/progress/progress_test.go b/sync_diff_inspector/progress/progress_test.go index 2bb416dcd..aadd8fc71 100644 --- a/sync_diff_inspector/progress/progress_test.go +++ b/sync_diff_inspector/progress/progress_test.go @@ -59,6 +59,9 @@ func TestTableError(t *testing.T) { p.StartTable("1", 50, true) p.RegisterTable("2", true, true, 0) p.StartTable("2", 1, true) + p.RegisterTable("3", true, true, -1) + p.StartTable("3", 1, true) + p.Inc("2") buffer := new(bytes.Buffer) p.SetOutput(buffer) @@ -73,6 +76,9 @@ func TestTableError(t *testing.T) { "\x1b[2A\x1b[JComparing the table structure of `2` ... failure\n"+ "_____________________________________________________________________________\n"+ "Progress [==============================>------------------------------] 50% 0/0\n"+ + "\x1b[2A\x1b[JComparing the table data of `3` ...skipped\n"+ + "_____________________________________________________________________________\n"+ + "Progress [=============================================>---------------] 75% 0/1\n"+ "\x1b[1A\x1b[J\nError in comparison process:\n[aaa]\n\n"+ "You can view the comparison details through './output_dir/sync_diff_inspector.log'\n", ) diff --git a/sync_diff_inspector/report/report_test.go b/sync_diff_inspector/report/report_test.go index ac9398843..0bfae286c 100644 --- a/sync_diff_inspector/report/report_test.go +++ b/sync_diff_inspector/report/report_test.go @@ -70,6 +70,12 @@ func TestReport(t *testing.T) { Info: tableInfo2, Collation: "[123]", }, + { + Schema: "dtest", + Table: "atbl", + Info: tableInfo2, + Collation: "[123]", + }, } configs := []*ReportConfig{ { @@ -118,26 +124,29 @@ func TestReport(t *testing.T) { require.True(t, result.DataEqual) require.True(t, result.StructEqual) - require.Equal(t, new_report.getSortedTables(), [][]string{{"`atest`.`atbl`", "0", "0"}, {"`ctest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) + require.Equal(t, new_report.getSortedTables(), [][]string{{"`atest`.`atbl`", "0", "0"}, {"`ctest`.`atbl`", "0", "0"}, {"`dtest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) require.Equal(t, new_report.getDiffRows(), [][]string{}) new_report.SetTableStructCheckResult("atest", "atbl", true, false, 0) new_report.SetTableDataCheckResult("atest", "atbl", false, 111, 222, 333, 333, &chunk.ChunkID{1, 1, 1, 1, 2}) - require.Equal(t, new_report.getSortedTables(), [][]string{{"`ctest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) + require.Equal(t, new_report.getSortedTables(), [][]string{{"`ctest`.`atbl`", "0", "0"}, {"`dtest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) require.Equal(t, new_report.getDiffRows(), [][]string{{"`atest`.`atbl`", "succeed", "true", "+111/-222", "333", "333"}}) new_report.SetTableStructCheckResult("atest", "atbl", false, false, 0) - require.Equal(t, new_report.getSortedTables(), [][]string{{"`ctest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) + require.Equal(t, new_report.getSortedTables(), [][]string{{"`ctest`.`atbl`", "0", "0"}, {"`dtest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) require.Equal(t, new_report.getDiffRows(), [][]string{{"`atest`.`atbl`", "succeed", "false", "+111/-222", "333", "333"}}) new_report.SetTableStructCheckResult("ctest", "atbl", false, true, 0) + new_report.SetTableStructCheckResult("dtest", "atbl", false, true, -1) + buf := new(bytes.Buffer) new_report.Print(buf) info := buf.String() require.Contains(t, info, "The structure of `atest`.`atbl` is not equal\n") require.Contains(t, info, "The data of `atest`.`atbl` is not equal\n") require.Contains(t, info, "The structure of `ctest`.`atbl` is not equal, and data-check is skipped\n") + require.Contains(t, info, "The data of `dtest`.`atbl` does not exist in downstream database\n") require.Contains(t, info, "\n"+ "The rest of tables are all equal.\n\n"+ "A total of 0 tables have been compared, 0 tables finished, 0 tables failed, 0 tables skipped.\n"+ diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index c960cee82..458e4ce7d 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -41,6 +41,7 @@ mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t4 valu mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t4 (a int, b int, primary key(a));" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log +check_contains "A total of 5 tables have been compared, 1 tables finished, 2 tables failed, 2 tables skipped" $OUT_DIR/table_skip_diff.output cat $OUT_DIR/summary.txt rm -rf $OUT_DIR/* From 54e2ab97dc5c6a41f8966191e35f044bf4c75221 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Thu, 5 Jan 2023 11:52:50 +0800 Subject: [PATCH 09/20] update --- sync_diff_inspector/report/report.go | 2 +- sync_diff_inspector/source/source.go | 10 ++++------ tests/sync_diff_inspector/table_skip/run.sh | 4 ++++ 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/sync_diff_inspector/report/report.go b/sync_diff_inspector/report/report.go index 9d33662ff..908167aac 100644 --- a/sync_diff_inspector/report/report.go +++ b/sync_diff_inspector/report/report.go @@ -80,7 +80,7 @@ type Report struct { Result string `json:"-"` // Result is pass or fail PassNum int32 `json:"-"` // The pass number of tables FailedNum int32 `json:"-"` // The failed number of tables - SkippedNum int32 `json:"-"` // Theskippeded number of tables + SkippedNum int32 `json:"-"` // The skipped number of tables TableResults map[string]map[string]*TableResult `json:"table-results"` // TableResult saved the map of `schema` => `table` => `tableResult` StartTime time.Time `json:"start-time"` Duration time.Duration `json:"time-duration"` diff --git a/sync_diff_inspector/source/source.go b/sync_diff_inspector/source/source.go index fa13d9e43..f931becf2 100644 --- a/sync_diff_inspector/source/source.go +++ b/sync_diff_inspector/source/source.go @@ -376,6 +376,7 @@ type RangeIterator interface { } func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]struct{}, sourceMap map[string]struct{}, skipNonExistingTable bool) ([]*common.TableDiff, error) { + tableIndexMap := getIndexMapForTable(tableDiffs) // check target exists but source not found for tableDiff := range targetMap { // target table have all passed in tableFilter @@ -383,7 +384,7 @@ func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]stru if !skipNonExistingTable { return tableDiffs, errors.Errorf("the source has no table to be compared. target-table is `%s`", tableDiff) } - index := getIndexMapForTable(tableDiffs, tableDiff)[tableDiff] + index := tableIndexMap[tableDiff] if tableDiffs[index].NeedSkippedTable == 0 { tableDiffs[index].NeedSkippedTable = common.UpstreamTableLackFlag log.Info("the source has no table to be compared", zap.String("target-table", tableDiff)) @@ -410,14 +411,11 @@ func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]stru return tableDiffs, nil } -// Get the index of table in tableDiffs by uniqueID:`schema`.`table` -func getIndexMapForTable(tableDiffs []*common.TableDiff, uniqueID string) map[string]int { +func getIndexMapForTable(tableDiffs []*common.TableDiff) map[string]int { tableIndexMap := make(map[string]int) for i := 0; i < len(tableDiffs); i++ { tableUniqueID := utils.UniqueID(tableDiffs[i].Schema, tableDiffs[i].Table) - if tableUniqueID == uniqueID { - tableIndexMap[uniqueID] = i - } + tableIndexMap[tableUniqueID] = i } return tableIndexMap } diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index 458e4ce7d..9601d696d 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -27,6 +27,10 @@ mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t3 (a int, b int, p mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t3 values (1,1);" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check pass" $OUT_DIR/sync_diff.log +check_contains "Comparing the table data of ``diff_test`.`t2`` ...skipped" $OUT_DIR/table_skip_diff.output +check_contains "Comparing the table data of ``diff_test`.`t3`` ...skipped" $OUT_DIR/table_skip_diff.output +check_contains "The data of `diff_test`.`t2` does not exist in downstream database" $OUT_DIR/table_skip_diff.output +check_contains "The data of `diff_test`.`t3` does not exist in upstream database" $OUT_DIR/table_skip_diff.output rm -rf $OUT_DIR/* echo "make some table data not equal" From 8a0f59f6cbe0f917cf15dc60badbbe7a76ebdde0 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Thu, 5 Jan 2023 13:21:18 +0800 Subject: [PATCH 10/20] update test --- tests/sync_diff_inspector/table_skip/run.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index 9601d696d..4fc64778a 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -27,10 +27,10 @@ mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t3 (a int, b int, p mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t3 values (1,1);" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check pass" $OUT_DIR/sync_diff.log -check_contains "Comparing the table data of ``diff_test`.`t2`` ...skipped" $OUT_DIR/table_skip_diff.output -check_contains "Comparing the table data of ``diff_test`.`t3`` ...skipped" $OUT_DIR/table_skip_diff.output -check_contains "The data of `diff_test`.`t2` does not exist in downstream database" $OUT_DIR/table_skip_diff.output -check_contains "The data of `diff_test`.`t3` does not exist in upstream database" $OUT_DIR/table_skip_diff.output +check_contains "Comparing the table data of \`\`diff_test\`.\`t2\`\` ...skipped" $OUT_DIR/table_skip_diff.output +check_contains "Comparing the table data of \`\`diff_test\`.\`t3\`\` ...skipped" $OUT_DIR/table_skip_diff.output +check_contains "The data of \`diff_test\`.\`t2\` does not exist in downstream database" $OUT_DIR/table_skip_diff.output +check_contains "The data of \`diff_test\`.\`t3\` does not exist in upstream database" $OUT_DIR/table_skip_diff.output rm -rf $OUT_DIR/* echo "make some table data not equal" From 47368645d536f05009ffb96ce4b85683059a470a Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Fri, 6 Jan 2023 12:16:36 +0800 Subject: [PATCH 11/20] update test --- sync_diff_inspector/diff.go | 5 +-- sync_diff_inspector/progress/progress_test.go | 27 ++++++++------ sync_diff_inspector/report/report.go | 37 +++++++------------ sync_diff_inspector/report/report_test.go | 30 +++++++-------- sync_diff_inspector/source/source.go | 5 +-- tests/sync_diff_inspector/table_skip/run.sh | 5 +++ 6 files changed, 52 insertions(+), 57 deletions(-) diff --git a/sync_diff_inspector/diff.go b/sync_diff_inspector/diff.go index fb43ba20e..c5675e2dc 100644 --- a/sync_diff_inspector/diff.go +++ b/sync_diff_inspector/diff.go @@ -297,16 +297,13 @@ func (df *Diff) StructEqual(ctx context.Context) error { tableIndex = df.startRange.ChunkRange.Index.TableIndex } for ; tableIndex < len(tables); tableIndex++ { - var isEqual, isSkip bool - isAllTableExist := tables[tableIndex].NeedSkippedTable + isEqual, isSkip, isAllTableExist := false, true, tables[tableIndex].NeedSkippedTable if source.AllTableExist(tables[tableIndex]) { var err error isEqual, isSkip, err = df.compareStruct(ctx, tableIndex) if err != nil { return errors.Trace(err) } - } else { - isEqual, isSkip = false, true } progress.RegisterTable(dbutil.TableName(tables[tableIndex].Schema, tables[tableIndex].Table), !isEqual, isSkip, isAllTableExist) df.report.SetTableStructCheckResult(tables[tableIndex].Schema, tables[tableIndex].Table, isEqual, isSkip, isAllTableExist) diff --git a/sync_diff_inspector/progress/progress_test.go b/sync_diff_inspector/progress/progress_test.go index aadd8fc71..bf25a9f50 100644 --- a/sync_diff_inspector/progress/progress_test.go +++ b/sync_diff_inspector/progress/progress_test.go @@ -19,17 +19,18 @@ import ( "testing" "time" + "github.com/pingcap/tidb-tools/sync_diff_inspector/source/common" "github.com/stretchr/testify/require" ) func TestProgress(t *testing.T) { - p := NewTableProgressPrinter(4, 0) - p.RegisterTable("1", true, true, 1) + p := NewTableProgressPrinter(6, 0) + p.RegisterTable("1", true, true, common.AllTableExistFlag) p.StartTable("1", 50, true) - p.RegisterTable("2", true, false, 0) + p.RegisterTable("2", true, false, common.AllTableExistFlag) p.StartTable("2", 2, true) p.Inc("2") - p.RegisterTable("3", false, false, 0) + p.RegisterTable("3", false, false, common.AllTableExistFlag) p.StartTable("3", 1, false) p.Inc("2") p.Inc("3") @@ -39,6 +40,10 @@ func TestProgress(t *testing.T) { p.FailTable("4") p.Inc("3") p.Inc("4") + p.RegisterTable("5", true, true, common.UpstreamTableLackFlag) + p.StartTable("5", 1, true) + p.RegisterTable("6", true, true, common.DownstreamTableLackFlag) + p.StartTable("6", 1, true) time.Sleep(500 * time.Millisecond) p.Close() buffer := new(bytes.Buffer) @@ -47,19 +52,19 @@ func TestProgress(t *testing.T) { require.Equal( t, buffer.String(), - "\x1b[1A\x1b[J\nSummary:\n\nThe data of `1` does not exist in upstream database.\nThe structure of `2` is not equal.\nThe data of `4` is not equal.\n"+ - "\nThe rest of the tables are all equal.\nA total of 4 tables have been compared, 1 tables finished, 2 tables failed, 1 tables skipped.\nThe patch file has been generated to './output_dir/patch.sql'\n"+ + "\x1b[1A\x1b[J\nSummary:\n\nThe structure of `1` is not equal.\nThe structure of `2` is not equal.\nThe data of `4` is not equal.\nThe data of `5` does not exist in upstream database.\nThe data of `6` does not exist in downstream database.\n"+ + "\nThe rest of the tables are all equal.\nA total of 6 tables have been compared, 1 tables finished, 3 tables failed, 2 tables skipped.\nThe patch file has been generated to './output_dir/patch.sql'\n"+ "You can view the comparison details through './output_dir/sync_diff_inspector.log'\n\n", ) } func TestTableError(t *testing.T) { p := NewTableProgressPrinter(4, 0) - p.RegisterTable("1", true, true, 0) + p.RegisterTable("1", true, true, common.AllTableExistFlag) p.StartTable("1", 50, true) - p.RegisterTable("2", true, true, 0) + p.RegisterTable("2", true, true, common.AllTableExistFlag) p.StartTable("2", 1, true) - p.RegisterTable("3", true, true, -1) + p.RegisterTable("3", true, true, common.DownstreamTableLackFlag) p.StartTable("3", 1, true) p.Inc("2") @@ -86,9 +91,9 @@ func TestTableError(t *testing.T) { func TestAllSuccess(t *testing.T) { Init(2, 0) - RegisterTable("1", false, false, 0) + RegisterTable("1", false, false, common.AllTableExistFlag) StartTable("1", 1, true) - RegisterTable("2", false, false, 0) + RegisterTable("2", false, false, common.AllTableExistFlag) StartTable("2", 1, true) Inc("1") Inc("2") diff --git a/sync_diff_inspector/report/report.go b/sync_diff_inspector/report/report.go index 908167aac..f6528640e 100644 --- a/sync_diff_inspector/report/report.go +++ b/sync_diff_inspector/report/report.go @@ -56,16 +56,16 @@ type ReportConfig struct { // TableResult saves the check result for every table. type TableResult struct { - Schema string `json:"schema"` - Table string `json:"table"` - StructEqual bool `json:"struct-equal"` - DataSkip bool `json:"data-skip"` - DataEqual bool `json:"data-equal"` - MeetError error `json:"-"` - ChunkMap map[string]*ChunkResult `json:"chunk-result"` // `ChunkMap` stores the `ChunkResult` of each chunk of the table - UpCount int64 `json:"up-count"` // `UpCount` is the number of rows in the table from upstream - DownCount int64 `json:"down-count"` // `DownCount` is the number of rows in the table from downstream - TableSkipped int `json:"table-skipped"` + Schema string `json:"schema"` + Table string `json:"table"` + StructEqual bool `json:"struct-equal"` + DataSkip bool `json:"data-skip"` + DataEqual bool `json:"data-equal"` + MeetError error `json:"-"` + ChunkMap map[string]*ChunkResult `json:"chunk-result"` // `ChunkMap` stores the `ChunkResult` of each chunk of the table + UpCount int64 `json:"up-count"` // `UpCount` is the number of rows in the table from upstream + DownCount int64 `json:"down-count"` // `DownCount` is the number of rows in the table from downstream + TableLack int `json:"table-lack"` } // ChunkResult save the necessarily information to provide summary information @@ -317,7 +317,7 @@ func (r *Report) SetTableStructCheckResult(schema, table string, equal bool, ski tableResult := r.TableResults[schema][table] tableResult.StructEqual = equal tableResult.DataSkip = skip - tableResult.TableSkipped = exist + tableResult.TableLack = exist if !equal && AllTableExist(tableResult) && r.Result != Error { r.Result = Fail } @@ -418,22 +418,13 @@ func (r *Report) GetSnapshot(chunkID *chunk.ChunkID, schema, table string) (*Rep } func AllTableExist(result *TableResult) bool { - if result.TableSkipped == common.AllTableExistFlag { - return true - } - return false + return result.TableLack == common.AllTableExistFlag } func UpstreamTableLack(result *TableResult) bool { - if result.TableSkipped == common.UpstreamTableLackFlag { - return true - } - return false + return result.TableLack == common.UpstreamTableLackFlag } func DownstreamTableLack(result *TableResult) bool { - if result.TableSkipped == common.DownstreamTableLackFlag { - return true - } - return false + return result.TableLack == common.DownstreamTableLackFlag } diff --git a/sync_diff_inspector/report/report_test.go b/sync_diff_inspector/report/report_test.go index 0bfae286c..9a3802009 100644 --- a/sync_diff_inspector/report/report_test.go +++ b/sync_diff_inspector/report/report_test.go @@ -110,7 +110,7 @@ func TestReport(t *testing.T) { report.CalculateTotalSize(ctx, db) // Test Table Report - report.SetTableStructCheckResult("test", "tbl", true, false, 0) + report.SetTableStructCheckResult("test", "tbl", true, false, common.AllTableExistFlag) report.SetTableDataCheckResult("test", "tbl", true, 100, 200, 222, 222, &chunk.ChunkID{1, 1, 1, 1, 2}) report.SetTableMeetError("test", "tbl", errors.New("eeee")) @@ -127,18 +127,18 @@ func TestReport(t *testing.T) { require.Equal(t, new_report.getSortedTables(), [][]string{{"`atest`.`atbl`", "0", "0"}, {"`ctest`.`atbl`", "0", "0"}, {"`dtest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) require.Equal(t, new_report.getDiffRows(), [][]string{}) - new_report.SetTableStructCheckResult("atest", "atbl", true, false, 0) + new_report.SetTableStructCheckResult("atest", "atbl", true, false, common.AllTableExistFlag) new_report.SetTableDataCheckResult("atest", "atbl", false, 111, 222, 333, 333, &chunk.ChunkID{1, 1, 1, 1, 2}) require.Equal(t, new_report.getSortedTables(), [][]string{{"`ctest`.`atbl`", "0", "0"}, {"`dtest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) require.Equal(t, new_report.getDiffRows(), [][]string{{"`atest`.`atbl`", "succeed", "true", "+111/-222", "333", "333"}}) - new_report.SetTableStructCheckResult("atest", "atbl", false, false, 0) + new_report.SetTableStructCheckResult("atest", "atbl", false, false, common.AllTableExistFlag) require.Equal(t, new_report.getSortedTables(), [][]string{{"`ctest`.`atbl`", "0", "0"}, {"`dtest`.`atbl`", "0", "0"}, {"`test`.`tbl`", "222", "222"}}) require.Equal(t, new_report.getDiffRows(), [][]string{{"`atest`.`atbl`", "succeed", "false", "+111/-222", "333", "333"}}) - new_report.SetTableStructCheckResult("ctest", "atbl", false, true, 0) + new_report.SetTableStructCheckResult("ctest", "atbl", false, true, common.AllTableExistFlag) - new_report.SetTableStructCheckResult("dtest", "atbl", false, true, -1) + new_report.SetTableStructCheckResult("dtest", "atbl", false, true, common.DownstreamTableLackFlag) buf := new(bytes.Buffer) new_report.Print(buf) @@ -255,7 +255,7 @@ func TestPrint(t *testing.T) { var buf *bytes.Buffer // All Pass - report.SetTableStructCheckResult("test", "tbl", true, false, 0) + report.SetTableStructCheckResult("test", "tbl", true, false, common.AllTableExistFlag) report.SetTableDataCheckResult("test", "tbl", true, 0, 0, 22, 22, &chunk.ChunkID{0, 0, 0, 0, 1}) buf = new(bytes.Buffer) report.Print(buf) @@ -264,7 +264,7 @@ func TestPrint(t *testing.T) { // Error report.SetTableMeetError("test", "tbl1", errors.New("123")) - report.SetTableStructCheckResult("test", "tbl1", false, false, 0) + report.SetTableStructCheckResult("test", "tbl1", false, false, common.AllTableExistFlag) buf = new(bytes.Buffer) report.Print(buf) require.Equal(t, buf.String(), "Error in comparison process:\n"+ @@ -329,17 +329,17 @@ func TestGetSnapshot(t *testing.T) { } report.Init(tableDiffs, configsBytes[:2], configsBytes[2]) - report.SetTableStructCheckResult("test", "tbl", true, false, 0) + report.SetTableStructCheckResult("test", "tbl", true, false, common.AllTableExistFlag) report.SetTableDataCheckResult("test", "tbl", false, 100, 100, 200, 300, &chunk.ChunkID{0, 0, 0, 1, 10}) report.SetTableDataCheckResult("test", "tbl", true, 0, 0, 300, 300, &chunk.ChunkID{0, 0, 0, 3, 10}) report.SetTableDataCheckResult("test", "tbl", false, 200, 200, 400, 500, &chunk.ChunkID{0, 0, 0, 3, 10}) - report.SetTableStructCheckResult("atest", "tbl", true, false, 0) + report.SetTableStructCheckResult("atest", "tbl", true, false, common.AllTableExistFlag) report.SetTableDataCheckResult("atest", "tbl", false, 100, 100, 500, 600, &chunk.ChunkID{0, 0, 0, 0, 10}) report.SetTableDataCheckResult("atest", "tbl", true, 0, 0, 600, 600, &chunk.ChunkID{0, 0, 0, 3, 10}) report.SetTableDataCheckResult("atest", "tbl", false, 200, 200, 700, 800, &chunk.ChunkID{0, 0, 0, 3, 10}) - report.SetTableStructCheckResult("xtest", "tbl", true, false, 0) + report.SetTableStructCheckResult("xtest", "tbl", true, false, common.AllTableExistFlag) report.SetTableDataCheckResult("xtest", "tbl", false, 100, 100, 800, 900, &chunk.ChunkID{0, 0, 0, 0, 10}) report.SetTableDataCheckResult("xtest", "tbl", true, 0, 0, 900, 900, &chunk.ChunkID{0, 0, 0, 1, 10}) report.SetTableDataCheckResult("xtest", "tbl", false, 200, 200, 1000, 1100, &chunk.ChunkID{0, 0, 0, 3, 10}) @@ -463,19 +463,19 @@ func TestCommitSummary(t *testing.T) { } report.Init(tableDiffs, configsBytes[:2], configsBytes[2]) - report.SetTableStructCheckResult("test", "tbl", true, false, 0) + report.SetTableStructCheckResult("test", "tbl", true, false, common.AllTableExistFlag) report.SetTableDataCheckResult("test", "tbl", true, 100, 200, 400, 400, &chunk.ChunkID{0, 0, 0, 1, 10}) - report.SetTableStructCheckResult("atest", "tbl", true, false, 0) + report.SetTableStructCheckResult("atest", "tbl", true, false, common.AllTableExistFlag) report.SetTableDataCheckResult("atest", "tbl", false, 100, 200, 500, 600, &chunk.ChunkID{0, 0, 0, 2, 10}) - report.SetTableStructCheckResult("xtest", "tbl", false, false, 0) + report.SetTableStructCheckResult("xtest", "tbl", false, false, common.AllTableExistFlag) report.SetTableDataCheckResult("xtest", "tbl", false, 100, 200, 600, 700, &chunk.ChunkID{0, 0, 0, 3, 10}) - report.SetTableStructCheckResult("xtest", "tb1", false, true, 1) + report.SetTableStructCheckResult("xtest", "tb1", false, true, common.UpstreamTableLackFlag) report.SetTableDataCheckResult("xtest", "tb1", false, 0, 200, 0, 200, &chunk.ChunkID{0, 0, 0, 4, 10}) - report.SetTableStructCheckResult("xtest", "tb2", false, true, -1) + report.SetTableStructCheckResult("xtest", "tb2", false, true, common.DownstreamTableLackFlag) report.SetTableDataCheckResult("xtest", "tb2", false, 100, 0, 100, 0, &chunk.ChunkID{0, 0, 0, 5, 10}) err = report.CommitSummary() diff --git a/sync_diff_inspector/source/source.go b/sync_diff_inspector/source/source.go index f931becf2..3db434828 100644 --- a/sync_diff_inspector/source/source.go +++ b/sync_diff_inspector/source/source.go @@ -421,8 +421,5 @@ func getIndexMapForTable(tableDiffs []*common.TableDiff) map[string]int { } func AllTableExist(tableDiffs *common.TableDiff) bool { - if tableDiffs.NeedSkippedTable == common.AllTableExistFlag { - return true - } - return false + return tableDiffs.NeedSkippedTable == common.AllTableExistFlag } diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index 4fc64778a..73331bab4 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -31,12 +31,16 @@ check_contains "Comparing the table data of \`\`diff_test\`.\`t2\`\` ...skipped" check_contains "Comparing the table data of \`\`diff_test\`.\`t3\`\` ...skipped" $OUT_DIR/table_skip_diff.output check_contains "The data of \`diff_test\`.\`t2\` does not exist in downstream database" $OUT_DIR/table_skip_diff.output check_contains "The data of \`diff_test\`.\`t3\` does not exist in upstream database" $OUT_DIR/table_skip_diff.output +check_contains "| TABLE | RESULT | STRUCTURE EQUALITY | DATA DIFF ROWS | UPCOUNT | DOWNCOUNT |" $OUT_DIR/summary.txt +check_contains "| \`diff_test\`.\`t2\` | skipped | false | +1/-0 | 1 | 0 |" $OUT_DIR/summary.txt +check_contains "| \`diff_test\`.\`t3\` | skipped | false | +0/-1 | 0 | 1 |" $OUT_DIR/summary.txt rm -rf $OUT_DIR/* echo "make some table data not equal" mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t1 values (2,2);" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log +check_contains "| \`diff_test\`.\`t1\` | succeed | true | +0/-1 | 1 | 2 |" $OUT_DIR/summary.txt rm -rf $OUT_DIR/* echo "make some table structure not equal" @@ -45,6 +49,7 @@ mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t4 valu mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t4 (a int, b int, primary key(a));" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log +check_contains "| \`diff_test\`.\`t4\` | succeed | false | +0/-0 | 0 | 0 |" $OUT_DIR/summary.txt check_contains "A total of 5 tables have been compared, 1 tables finished, 2 tables failed, 2 tables skipped" $OUT_DIR/table_skip_diff.output cat $OUT_DIR/summary.txt rm -rf $OUT_DIR/* From 3f254c8835a7caeadca372d106ee434bb9850e13 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Mon, 16 Jan 2023 10:56:15 +0800 Subject: [PATCH 12/20] fix --- sync_diff_inspector/progress/progress.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sync_diff_inspector/progress/progress.go b/sync_diff_inspector/progress/progress.go index 827b6c8f7..c8f2176a8 100644 --- a/sync_diff_inspector/progress/progress.go +++ b/sync_diff_inspector/progress/progress.go @@ -135,11 +135,12 @@ func (tpp *TableProgressPrinter) RegisterTable(name string, isFailed bool, isDon var state table_state_t if isFailed { if isDone { - if isExist == common.UpstreamTableLackFlag { + switch isExist { + case common.UpstreamTableLackFlag: state = TABLE_STATE_NOT_EXSIT_UPSTREAM | TABLE_STATE_REGISTER - } else if isExist == common.DownstreamTableLackFlag { + case common.DownstreamTableLackFlag: state = TABLE_STATE_NOT_EXSIT_DOWNSTREAM | TABLE_STATE_REGISTER - } else { + default: state = TABLE_STATE_RESULT_FAIL_STRUCTURE_DONE | TABLE_STATE_REGISTER } } else { From 5af05c2e166bcbbdf0884112a20e54821e747f49 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Thu, 19 Jan 2023 02:30:03 +0800 Subject: [PATCH 13/20] fix code --- sync_diff_inspector/diff.go | 10 +-- sync_diff_inspector/report/report.go | 31 +++----- sync_diff_inspector/source/chunks_iter.go | 2 +- .../source/common/table_diff.go | 12 ++-- sync_diff_inspector/source/mysql_shard.go | 70 ++++++++++--------- sync_diff_inspector/source/source.go | 14 ++-- sync_diff_inspector/source/source_test.go | 6 +- sync_diff_inspector/source/tidb.go | 17 ++++- .../table_skip/config_router.toml | 66 +++++++++++++++++ tests/sync_diff_inspector/table_skip/data.sql | 2 +- tests/sync_diff_inspector/table_skip/run.sh | 13 +++- 11 files changed, 162 insertions(+), 81 deletions(-) create mode 100644 tests/sync_diff_inspector/table_skip/config_router.toml diff --git a/sync_diff_inspector/diff.go b/sync_diff_inspector/diff.go index c5675e2dc..e6708fb83 100644 --- a/sync_diff_inspector/diff.go +++ b/sync_diff_inspector/diff.go @@ -297,8 +297,8 @@ func (df *Diff) StructEqual(ctx context.Context) error { tableIndex = df.startRange.ChunkRange.Index.TableIndex } for ; tableIndex < len(tables); tableIndex++ { - isEqual, isSkip, isAllTableExist := false, true, tables[tableIndex].NeedSkippedTable - if source.AllTableExist(tables[tableIndex]) { + isEqual, isSkip, isAllTableExist := false, true, tables[tableIndex].TableLack + if common.AllTableExist(tables[tableIndex].TableLack) { var err error isEqual, isSkip, err = df.compareStruct(ctx, tableIndex) if err != nil { @@ -421,9 +421,9 @@ func (df *Diff) consume(ctx context.Context, rangeInfo *splitter.RangeInfo) bool if rangeInfo.ChunkRange.Type == chunk.Empty { dml.node.State = checkpoints.IgnoreState // for tables that don't exist upstream or downstream - if !source.AllTableExist(tableDiff) { - upCount, _ := dbutil.GetRowCount(ctx, df.upstream.GetDB(), schema, table, "", nil) - downCount, _ := dbutil.GetRowCount(ctx, df.downstream.GetDB(), schema, table, "", nil) + if !common.AllTableExist(tableDiff.TableLack) { + upCount := df.upstream.GetCountAndCrc32(ctx, rangeInfo).Count + downCount := df.downstream.GetCountAndCrc32(ctx, rangeInfo).Count df.report.SetTableDataCheckResult(schema, table, false, int(upCount), int(downCount), upCount, downCount, id) return false } diff --git a/sync_diff_inspector/report/report.go b/sync_diff_inspector/report/report.go index f6528640e..5e3aa208a 100644 --- a/sync_diff_inspector/report/report.go +++ b/sync_diff_inspector/report/report.go @@ -132,7 +132,7 @@ func (r *Report) getDiffRows() [][]string { } diffRow := make([]string, 0) diffRow = append(diffRow, dbutil.TableName(schema, table)) - if !AllTableExist(result) { + if !common.AllTableExist(result.TableLack) { diffRow = append(diffRow, "skipped") } else { diffRow = append(diffRow, "succeed") @@ -176,7 +176,7 @@ func (r *Report) CommitSummary() error { for _, result := range tableMap { if result.StructEqual && result.DataEqual { passNum++ - } else if !AllTableExist(result) { + } else if !common.AllTableExist(result.TableLack) { skippedNum++ } else { failedNum++ @@ -244,18 +244,19 @@ func (r *Report) Print(w io.Writer) error { for table, result := range tableMap { if !result.StructEqual { if result.DataSkip { - if UpstreamTableLack(result) { + switch result.TableLack { + case common.UpstreamTableLackFlag: summary.WriteString(fmt.Sprintf("The data of %s does not exist in upstream database\n", dbutil.TableName(schema, table))) - } else if DownstreamTableLack(result) { + case common.DownstreamTableLackFlag: summary.WriteString(fmt.Sprintf("The data of %s does not exist in downstream database\n", dbutil.TableName(schema, table))) - } else { + default: summary.WriteString(fmt.Sprintf("The structure of %s is not equal, and data-check is skipped\n", dbutil.TableName(schema, table))) } } else { summary.WriteString(fmt.Sprintf("The structure of %s is not equal\n", dbutil.TableName(schema, table))) } } - if !result.DataEqual && AllTableExist(result) { + if !result.DataEqual && common.AllTableExist(result.TableLack) { summary.WriteString(fmt.Sprintf("The data of %s is not equal\n", dbutil.TableName(schema, table))) } } @@ -318,7 +319,7 @@ func (r *Report) SetTableStructCheckResult(schema, table string, equal bool, ski tableResult.StructEqual = equal tableResult.DataSkip = skip tableResult.TableLack = exist - if !equal && AllTableExist(tableResult) && r.Result != Error { + if !equal && common.AllTableExist(tableResult.TableLack) && r.Result != Error { r.Result = Fail } } @@ -340,11 +341,11 @@ func (r *Report) SetTableDataCheckResult(schema, table string, equal bool, rowsA } result.ChunkMap[id.ToString()].RowsAdd += rowsAdd result.ChunkMap[id.ToString()].RowsDelete += rowsDelete - if r.Result != Error && AllTableExist(result) { + if r.Result != Error && common.AllTableExist(result.TableLack) { r.Result = Fail } } - if !equal && AllTableExist(result) && r.Result != Error { + if !equal && common.AllTableExist(result.TableLack) && r.Result != Error { r.Result = Fail } } @@ -416,15 +417,3 @@ func (r *Report) GetSnapshot(chunkID *chunk.ChunkID, schema, table string) (*Rep task: task, }, nil } - -func AllTableExist(result *TableResult) bool { - return result.TableLack == common.AllTableExistFlag -} - -func UpstreamTableLack(result *TableResult) bool { - return result.TableLack == common.UpstreamTableLackFlag -} - -func DownstreamTableLack(result *TableResult) bool { - return result.TableLack == common.DownstreamTableLackFlag -} diff --git a/sync_diff_inspector/source/chunks_iter.go b/sync_diff_inspector/source/chunks_iter.go index 0268b7ad3..ffdfb4de5 100644 --- a/sync_diff_inspector/source/chunks_iter.go +++ b/sync_diff_inspector/source/chunks_iter.go @@ -103,7 +103,7 @@ func (t *ChunksIterator) produceChunks(ctx context.Context, startRange *splitter for ; t.nextTableIndex < len(t.TableDiffs); t.nextTableIndex++ { curTableIndex := t.nextTableIndex // skip data-check, but still need to send a empty chunk to make checkpoint continuous - if t.TableDiffs[curTableIndex].IgnoreDataCheck || !AllTableExist(t.TableDiffs[curTableIndex]) { + if t.TableDiffs[curTableIndex].IgnoreDataCheck || !common.AllTableExist(t.TableDiffs[curTableIndex].TableLack) { pool.Apply(func() { table := t.TableDiffs[curTableIndex] progressID := dbutil.TableName(table.Schema, table.Table) diff --git a/sync_diff_inspector/source/common/table_diff.go b/sync_diff_inspector/source/common/table_diff.go index 1320a3cde..8f8588aa2 100644 --- a/sync_diff_inspector/source/common/table_diff.go +++ b/sync_diff_inspector/source/common/table_diff.go @@ -66,10 +66,10 @@ type TableDiff struct { ChunkSize int64 `json:"chunk-size"` - // NeedSkippedTable = 1: the table only exists downstream, - // NeedSkippedTable = -1: the table only exists upstream, - // NeedSkippedTable = 0: the table exists both upstream and downstream. - NeedSkippedTable int `json:"-"` + // TableLack = 1: the table only exists downstream, + // TableLack = -1: the table only exists upstream, + // TableLack = 0: the table exists both upstream and downstream. + TableLack int `json:"-"` } const ( @@ -77,3 +77,7 @@ const ( DownstreamTableLackFlag = -1 UpstreamTableLackFlag = 1 ) + +func AllTableExist(tableLack int) bool { + return tableLack == AllTableExistFlag +} diff --git a/sync_diff_inspector/source/mysql_shard.go b/sync_diff_inspector/source/mysql_shard.go index ac1477516..2926978ca 100644 --- a/sync_diff_inspector/source/mysql_shard.go +++ b/sync_diff_inspector/source/mysql_shard.go @@ -69,7 +69,7 @@ func getMatchedSourcesForTable(sourceTablesMap map[string][]*common.TableShardSo log.Fatal("unreachable, source tables map shouldn't be nil.") } matchSources, ok := sourceTablesMap[utils.UniqueID(table.Schema, table.Table)] - if !ok { + if !ok && common.AllTableExist(table.TableLack) { log.Fatal("unreachable, no match source tables in mysql shard source.") } return matchSources @@ -97,42 +97,44 @@ func (s *MySQLSources) GetCountAndCrc32(ctx context.Context, tableRange *splitte beginTime := time.Now() table := s.tableDiffs[tableRange.GetTableIndex()] chunk := tableRange.GetChunk() - - // for tables that do not exist upstream or downstream - if !AllTableExist(table) { - return &ChecksumInfo{ - Count: 0, - } - } - matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) - infoCh := make(chan *ChecksumInfo, len(s.sourceTablesMap)) - - for _, ms := range matchSources { - go func(ms *common.TableShardSource) { - count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, table.Info, chunk.Where, chunk.Args) - infoCh <- &ChecksumInfo{ - Checksum: checksum, - Count: count, - Err: err, - } - }(ms) - } - defer close(infoCh) - var ( err error totalCount int64 totalChecksum int64 ) + matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) + + if common.AllTableExist(table.TableLack) { + infoCh := make(chan *ChecksumInfo, len(s.sourceTablesMap)) + for _, ms := range matchSources { + go func(ms *common.TableShardSource) { + count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, table.Info, chunk.Where, chunk.Args) + infoCh <- &ChecksumInfo{ + Checksum: checksum, + Count: count, + Err: err, + } + }(ms) + } + defer close(infoCh) - for range matchSources { - info := <-infoCh - // catch the first error - if err == nil && info.Err != nil { - err = info.Err + for range matchSources { + info := <-infoCh + // catch the first error + if err == nil && info.Err != nil { + err = info.Err + } + totalCount += info.Count + totalChecksum ^= info.Checksum + } + } else { + var count int64 + if matchSources != nil { + for _, ms := range matchSources { + count, err = dbutil.GetRowCount(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, "", nil) + totalCount += count + } } - totalCount += info.Count - totalChecksum ^= info.Checksum } cost := time.Since(beginTime) @@ -169,7 +171,7 @@ func (s *MySQLSources) GetRowsIterator(ctx context.Context, tableRange *splitter table := s.tableDiffs[tableRange.GetTableIndex()] // for tables that do not exist upstream or downstream - if !AllTableExist(table) { + if !common.AllTableExist(table.TableLack) { return nil, nil } matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) @@ -234,7 +236,7 @@ func (s *MySQLSources) GetSnapshot() string { func (s *MySQLSources) GetSourceStructInfo(ctx context.Context, tableIndex int) ([]*model.TableInfo, error) { tableDiff := s.GetTables()[tableIndex] // for tables that do not exist upstream or downstream - if !AllTableExist(tableDiff) { + if !common.AllTableExist(tableDiff.TableLack) { return nil, nil } tableSources := getMatchedSourcesForTable(s.sourceTablesMap, tableDiff) @@ -334,12 +336,14 @@ func NewMySQLSources(ctx context.Context, tableDiffs []*common.TableDiff, ds []* } } uniqueId := utils.UniqueID(targetSchema, targetTable) + var isMatched bool // get all tables from all source db instance if f.MatchTable(targetSchema, targetTable) { // if match the filter, we should respect it and check target has this table later. sourceTablesAfterRoute[uniqueId] = struct{}{} + isMatched = true } - if _, ok := targetUniqueTableMap[uniqueId]; !ok { + if _, ok := targetUniqueTableMap[uniqueId]; !ok && !(isMatched && skipNonExistingTable) { continue } maxSourceRouteTableCount[uniqueId]++ diff --git a/sync_diff_inspector/source/source.go b/sync_diff_inspector/source/source.go index 3db434828..a4bc1061b 100644 --- a/sync_diff_inspector/source/source.go +++ b/sync_diff_inspector/source/source.go @@ -385,8 +385,8 @@ func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]stru return tableDiffs, errors.Errorf("the source has no table to be compared. target-table is `%s`", tableDiff) } index := tableIndexMap[tableDiff] - if tableDiffs[index].NeedSkippedTable == 0 { - tableDiffs[index].NeedSkippedTable = common.UpstreamTableLackFlag + if tableDiffs[index].TableLack == 0 { + tableDiffs[index].TableLack = common.UpstreamTableLackFlag log.Info("the source has no table to be compared", zap.String("target-table", tableDiff)) } } @@ -400,9 +400,9 @@ func checkTableMatched(tableDiffs []*common.TableDiff, targetMap map[string]stru } slice := strings.Split(strings.Replace(tableDiff, "`", "", -1), ".") tableDiffs = append(tableDiffs, &common.TableDiff{ - Schema: slice[0], - Table: slice[1], - NeedSkippedTable: common.DownstreamTableLackFlag, + Schema: slice[0], + Table: slice[1], + TableLack: common.DownstreamTableLackFlag, }) log.Info("the target has no table to be compared", zap.String("source-table", tableDiff)) } @@ -419,7 +419,3 @@ func getIndexMapForTable(tableDiffs []*common.TableDiff) map[string]int { } return tableIndexMap } - -func AllTableExist(tableDiffs *common.TableDiff) bool { - return tableDiffs.NeedSkippedTable == common.AllTableExistFlag -} diff --git a/sync_diff_inspector/source/source_test.go b/sync_diff_inspector/source/source_test.go index c6d3302a1..0a7a1964d 100644 --- a/sync_diff_inspector/source/source_test.go +++ b/sync_diff_inspector/source/source_test.go @@ -942,7 +942,7 @@ func TestCheckTableMatched(t *testing.T) { tables, err = checkTableMatched(tableDiffs, tmap, smap, true) require.NoError(t, err) - require.Equal(t, 0, tables[0].NeedSkippedTable) - require.Equal(t, 1, tables[1].NeedSkippedTable) - require.Equal(t, -1, tables[2].NeedSkippedTable) + require.Equal(t, 0, tables[0].TableLack) + require.Equal(t, 1, tables[1].TableLack) + require.Equal(t, -1, tables[2].TableLack) } diff --git a/sync_diff_inspector/source/tidb.go b/sync_diff_inspector/source/tidb.go index 2a403b48e..a467165da 100644 --- a/sync_diff_inspector/source/tidb.go +++ b/sync_diff_inspector/source/tidb.go @@ -124,9 +124,20 @@ func (s *TiDBSource) GetCountAndCrc32(ctx context.Context, tableRange *splitter. beginTime := time.Now() table := s.tableDiffs[tableRange.GetTableIndex()] chunk := tableRange.GetChunk() + var ( + checksum int64 + count int64 + err error + ) matchSource := getMatchSource(s.sourceTableMap, table) - count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, s.dbConn, matchSource.OriginSchema, matchSource.OriginTable, table.Info, chunk.Where, chunk.Args) + if common.AllTableExist(table.TableLack) { + count, checksum, err = utils.GetCountAndCRC32Checksum(ctx, s.dbConn, matchSource.OriginSchema, matchSource.OriginTable, table.Info, chunk.Where, chunk.Args) + } else { + if matchSource != nil { + count, err = dbutil.GetRowCount(ctx, s.dbConn, matchSource.OriginSchema, matchSource.OriginTable, "", nil) + } + } cost := time.Since(beginTime) return &ChecksumInfo{ @@ -236,11 +247,13 @@ func NewTiDBSource(ctx context.Context, tableDiffs []*common.TableDiff, ds *conf } uniqueId := utils.UniqueID(targetSchema, targetTable) + var isMatched bool if f.MatchTable(targetSchema, targetTable) { // if match the filter, we should respect it and check target has this table later. sourceTablesAfterRoute[uniqueId] = struct{}{} + isMatched = true } - if _, ok := targetUniqueTableMap[uniqueId]; ok { + if _, ok := targetUniqueTableMap[uniqueId]; ok || (isMatched && skipNonExistingTable) { if _, ok := sourceTableMap[uniqueId]; ok { log.Error("TiDB source don't support compare multiple source tables with one downstream table," + " if this happening when diff on same instance is fine. otherwise we are not guarantee this diff result is right") diff --git a/tests/sync_diff_inspector/table_skip/config_router.toml b/tests/sync_diff_inspector/table_skip/config_router.toml new file mode 100644 index 000000000..2d22f2ec6 --- /dev/null +++ b/tests/sync_diff_inspector/table_skip/config_router.toml @@ -0,0 +1,66 @@ +# Diff Configuration. + +######################### Global config ######################### + +# The number of goroutines created to check data. The number of connections between upstream and downstream databases are slightly greater than this value +check-thread-count = 4 + +# If enabled, SQL statements is exported to fix inconsistent tables +export-fix-sql = true + +# Only compares the table structure instead of the data +check-struct-only = false + +# skip validation for tables that don't exist upstream or downstream +skip-non-existing-table = true +######################### Datasource config ######################### +[data-sources.mysql1] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" + + route-rules = ["rule1"] + +[data-sources.mysql2] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" + + route-rules = ["rule2","rule3"] + +[data-sources.tidb0] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + +########################### Routes ########################### +[routes.rule1] +schema-pattern = "diff_test" # Matches the schema name of the data source. Supports the wildcards "*" and "?" +table-pattern = "t[1-2]" # Matches the table name of the data source. Supports the wildcards "*" and "?" +target-schema = "diff_test" # The name of the schema in the target database +target-table = "t5" # The name of the target table + +[routes.rule2] +schema-pattern = "diff_test" +table-pattern = "t0" +target-schema = "diff_test" +target-table = "t5" + +[routes.rule3] +schema-pattern = "diff_test" +table-pattern = "t4" +target-schema = "diff_test" +target-table = "t4" +######################### Task config ######################### +[task] + output-dir = "./output" + + source-instances = ["mysql1", "mysql2"] + + target-instance = "tidb0" + + # The tables of downstream databases to be compared. Each table needs to contain the schema name and the table name, separated by '.' + target-check-tables = ["diff_test.t5","diff_test.t4"] \ No newline at end of file diff --git a/tests/sync_diff_inspector/table_skip/data.sql b/tests/sync_diff_inspector/table_skip/data.sql index 9e8e3230e..7be211251 100644 --- a/tests/sync_diff_inspector/table_skip/data.sql +++ b/tests/sync_diff_inspector/table_skip/data.sql @@ -3,4 +3,4 @@ create database diff_test; create table diff_test.t0 (a int, b int, primary key(a)); create table diff_test.t1 (a int, b int, primary key(a)); insert into diff_test.t0 values (1,1); -insert into diff_test.t1 values (1,1); \ No newline at end of file +insert into diff_test.t1 values (2,2); \ No newline at end of file diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index 73331bab4..3292a0ffa 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -22,7 +22,7 @@ rm -rf $OUT_DIR/* echo "make some tables exist only upstream or downstream" mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "create table diff_test.t2 (a int, b int, primary key(a));" -mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t2 values (1,1);" +mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t2 values (3,3);" mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t3 (a int, b int, primary key(a));" mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t3 values (1,1);" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true @@ -37,7 +37,7 @@ check_contains "| \`diff_test\`.\`t3\` | skipped | false | +0/-1 rm -rf $OUT_DIR/* echo "make some table data not equal" -mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t1 values (2,2);" +mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t1 values (4,4);" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log check_contains "| \`diff_test\`.\`t1\` | succeed | true | +0/-1 | 1 | 2 |" $OUT_DIR/summary.txt @@ -54,4 +54,13 @@ check_contains "A total of 5 tables have been compared, 1 tables finished, 2 tab cat $OUT_DIR/summary.txt rm -rf $OUT_DIR/* +echo "test router case" +sed "s/\"127.0.0.1\"#MYSQL_HOST/\"${MYSQL_HOST}\"/g" ./config_router.toml | sed "s/3306#MYSQL_PORT/${MYSQL_PORT}/g" > ./config.toml +sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true +check_contains "check failed" $OUT_DIR/sync_diff.log +check_contains "| \`diff_test\`.\`t5\` | skipped | false | +3/-0 | 3 | 0 |" $OUT_DIR/summary.txt +check_contains "The data of \`diff_test\`.\`t5\` does not exist in downstream database" $OUT_DIR/table_skip_diff.output +check_contains "A total of 2 tables have been compared, 0 tables finished, 1 tables failed, 1 tables skipped" $OUT_DIR/table_skip_diff.output +rm -rf $OUT_DIR/* + echo "table_skip test passed" \ No newline at end of file From b928386ff86e380103311b9672c95ac782f15c89 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Thu, 19 Jan 2023 20:52:33 +0800 Subject: [PATCH 14/20] fix --- tests/sync_diff_inspector/table_skip/run.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index 3292a0ffa..a7e8c589b 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -55,8 +55,7 @@ cat $OUT_DIR/summary.txt rm -rf $OUT_DIR/* echo "test router case" -sed "s/\"127.0.0.1\"#MYSQL_HOST/\"${MYSQL_HOST}\"/g" ./config_router.toml | sed "s/3306#MYSQL_PORT/${MYSQL_PORT}/g" > ./config.toml -sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true +sync_diff_inspector --config=./config_router.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log check_contains "| \`diff_test\`.\`t5\` | skipped | false | +3/-0 | 3 | 0 |" $OUT_DIR/summary.txt check_contains "The data of \`diff_test\`.\`t5\` does not exist in downstream database" $OUT_DIR/table_skip_diff.output From a8b82a932ace582f160915e8e8fd1abc79d6c5df Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Thu, 19 Jan 2023 21:06:45 +0800 Subject: [PATCH 15/20] fix --- tests/sync_diff_inspector/table_skip/config_router.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/sync_diff_inspector/table_skip/config_router.toml b/tests/sync_diff_inspector/table_skip/config_router.toml index 2d22f2ec6..9195e585b 100644 --- a/tests/sync_diff_inspector/table_skip/config_router.toml +++ b/tests/sync_diff_inspector/table_skip/config_router.toml @@ -56,7 +56,7 @@ target-schema = "diff_test" target-table = "t4" ######################### Task config ######################### [task] - output-dir = "./output" + output-dir = "/tmp/tidb_tools_test/sync_diff_inspector/output" source-instances = ["mysql1", "mysql2"] From 34293bb67f87a72041106503cf6bfcebee520d9c Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Sun, 29 Jan 2023 17:29:03 +0800 Subject: [PATCH 16/20] add function `GetCountForLackTable` --- sync_diff_inspector/diff.go | 6 +- sync_diff_inspector/source/mysql_shard.go | 74 ++++++++++++----------- sync_diff_inspector/source/source.go | 3 + sync_diff_inspector/source/tidb.go | 29 ++++----- 4 files changed, 58 insertions(+), 54 deletions(-) diff --git a/sync_diff_inspector/diff.go b/sync_diff_inspector/diff.go index e6708fb83..e5b5e9f12 100644 --- a/sync_diff_inspector/diff.go +++ b/sync_diff_inspector/diff.go @@ -298,7 +298,7 @@ func (df *Diff) StructEqual(ctx context.Context) error { } for ; tableIndex < len(tables); tableIndex++ { isEqual, isSkip, isAllTableExist := false, true, tables[tableIndex].TableLack - if common.AllTableExist(tables[tableIndex].TableLack) { + if common.AllTableExist(isAllTableExist) { var err error isEqual, isSkip, err = df.compareStruct(ctx, tableIndex) if err != nil { @@ -422,8 +422,8 @@ func (df *Diff) consume(ctx context.Context, rangeInfo *splitter.RangeInfo) bool dml.node.State = checkpoints.IgnoreState // for tables that don't exist upstream or downstream if !common.AllTableExist(tableDiff.TableLack) { - upCount := df.upstream.GetCountAndCrc32(ctx, rangeInfo).Count - downCount := df.downstream.GetCountAndCrc32(ctx, rangeInfo).Count + upCount := df.upstream.GetCountForLackTable(ctx, rangeInfo) + downCount := df.downstream.GetCountForLackTable(ctx, rangeInfo) df.report.SetTableDataCheckResult(schema, table, false, int(upCount), int(downCount), upCount, downCount, id) return false } diff --git a/sync_diff_inspector/source/mysql_shard.go b/sync_diff_inspector/source/mysql_shard.go index 2926978ca..26789f289 100644 --- a/sync_diff_inspector/source/mysql_shard.go +++ b/sync_diff_inspector/source/mysql_shard.go @@ -97,44 +97,36 @@ func (s *MySQLSources) GetCountAndCrc32(ctx context.Context, tableRange *splitte beginTime := time.Now() table := s.tableDiffs[tableRange.GetTableIndex()] chunk := tableRange.GetChunk() + + matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) + infoCh := make(chan *ChecksumInfo, len(s.sourceTablesMap)) + + for _, ms := range matchSources { + go func(ms *common.TableShardSource) { + count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, table.Info, chunk.Where, chunk.Args) + infoCh <- &ChecksumInfo{ + Checksum: checksum, + Count: count, + Err: err, + } + }(ms) + } + defer close(infoCh) + var ( err error totalCount int64 totalChecksum int64 ) - matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) - if common.AllTableExist(table.TableLack) { - infoCh := make(chan *ChecksumInfo, len(s.sourceTablesMap)) - for _, ms := range matchSources { - go func(ms *common.TableShardSource) { - count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, table.Info, chunk.Where, chunk.Args) - infoCh <- &ChecksumInfo{ - Checksum: checksum, - Count: count, - Err: err, - } - }(ms) - } - defer close(infoCh) - - for range matchSources { - info := <-infoCh - // catch the first error - if err == nil && info.Err != nil { - err = info.Err - } - totalCount += info.Count - totalChecksum ^= info.Checksum - } - } else { - var count int64 - if matchSources != nil { - for _, ms := range matchSources { - count, err = dbutil.GetRowCount(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, "", nil) - totalCount += count - } + for range matchSources { + info := <-infoCh + // catch the first error + if err == nil && info.Err != nil { + err = info.Err } + totalCount += info.Count + totalChecksum ^= info.Checksum } cost := time.Since(beginTime) @@ -146,6 +138,20 @@ func (s *MySQLSources) GetCountAndCrc32(ctx context.Context, tableRange *splitte } } +func (s *MySQLSources) GetCountForLackTable(ctx context.Context, tableRange *splitter.RangeInfo) int64 { + table := s.tableDiffs[tableRange.GetTableIndex()] + var totalCount int64 + + matchSources := getMatchedSourcesForTable(s.sourceTablesMap, table) + if matchSources != nil { + for _, ms := range matchSources { + count, _ := dbutil.GetRowCount(ctx, ms.DBConn, ms.OriginSchema, ms.OriginTable, "", nil) + totalCount += count + } + } + return totalCount +} + func (s *MySQLSources) GetTables() []*common.TableDiff { return s.tableDiffs } @@ -336,12 +342,10 @@ func NewMySQLSources(ctx context.Context, tableDiffs []*common.TableDiff, ds []* } } uniqueId := utils.UniqueID(targetSchema, targetTable) - var isMatched bool - // get all tables from all source db instance - if f.MatchTable(targetSchema, targetTable) { + isMatched := f.MatchTable(targetSchema, targetTable) + if isMatched { // if match the filter, we should respect it and check target has this table later. sourceTablesAfterRoute[uniqueId] = struct{}{} - isMatched = true } if _, ok := targetUniqueTableMap[uniqueId]; !ok && !(isMatched && skipNonExistingTable) { continue diff --git a/sync_diff_inspector/source/source.go b/sync_diff_inspector/source/source.go index a4bc1061b..a795d2a82 100644 --- a/sync_diff_inspector/source/source.go +++ b/sync_diff_inspector/source/source.go @@ -85,6 +85,9 @@ type Source interface { // GetCountAndCrc32 gets the crc32 result and the count from given range. GetCountAndCrc32(context.Context, *splitter.RangeInfo) *ChecksumInfo + // GetCountForLackTable gets the count for tables that don't exist upstream or downstream. + GetCountForLackTable(context.Context, *splitter.RangeInfo) int64 + // GetRowsIterator gets the row data iterator from given range. GetRowsIterator(context.Context, *splitter.RangeInfo) (RowDataIterator, error) diff --git a/sync_diff_inspector/source/tidb.go b/sync_diff_inspector/source/tidb.go index a467165da..e8d2c5677 100644 --- a/sync_diff_inspector/source/tidb.go +++ b/sync_diff_inspector/source/tidb.go @@ -124,20 +124,8 @@ func (s *TiDBSource) GetCountAndCrc32(ctx context.Context, tableRange *splitter. beginTime := time.Now() table := s.tableDiffs[tableRange.GetTableIndex()] chunk := tableRange.GetChunk() - var ( - checksum int64 - count int64 - err error - ) - matchSource := getMatchSource(s.sourceTableMap, table) - if common.AllTableExist(table.TableLack) { - count, checksum, err = utils.GetCountAndCRC32Checksum(ctx, s.dbConn, matchSource.OriginSchema, matchSource.OriginTable, table.Info, chunk.Where, chunk.Args) - } else { - if matchSource != nil { - count, err = dbutil.GetRowCount(ctx, s.dbConn, matchSource.OriginSchema, matchSource.OriginTable, "", nil) - } - } + count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, s.dbConn, matchSource.OriginSchema, matchSource.OriginTable, table.Info, chunk.Where, chunk.Args) cost := time.Since(beginTime) return &ChecksumInfo{ @@ -148,6 +136,16 @@ func (s *TiDBSource) GetCountAndCrc32(ctx context.Context, tableRange *splitter. } } +func (s *TiDBSource) GetCountForLackTable(ctx context.Context, tableRange *splitter.RangeInfo) int64 { + table := s.tableDiffs[tableRange.GetTableIndex()] + matchSource := getMatchSource(s.sourceTableMap, table) + if matchSource != nil { + count, _ := dbutil.GetRowCount(ctx, s.dbConn, matchSource.OriginSchema, matchSource.OriginTable, "", nil) + return count + } + return 0 +} + func (s *TiDBSource) GetTables() []*common.TableDiff { return s.tableDiffs } @@ -247,11 +245,10 @@ func NewTiDBSource(ctx context.Context, tableDiffs []*common.TableDiff, ds *conf } uniqueId := utils.UniqueID(targetSchema, targetTable) - var isMatched bool - if f.MatchTable(targetSchema, targetTable) { + isMatched := f.MatchTable(targetSchema, targetTable) + if isMatched { // if match the filter, we should respect it and check target has this table later. sourceTablesAfterRoute[uniqueId] = struct{}{} - isMatched = true } if _, ok := targetUniqueTableMap[uniqueId]; ok || (isMatched && skipNonExistingTable) { if _, ok := sourceTableMap[uniqueId]; ok { From 03b86f3a5231c76bc59013e0291a134094a80b63 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Mon, 30 Jan 2023 12:03:12 +0800 Subject: [PATCH 17/20] fix test --- sync_diff_inspector/source/tidb.go | 1 + tests/sync_diff_inspector/table_skip/data.sql | 3 +-- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sync_diff_inspector/source/tidb.go b/sync_diff_inspector/source/tidb.go index e8d2c5677..8c47bf060 100644 --- a/sync_diff_inspector/source/tidb.go +++ b/sync_diff_inspector/source/tidb.go @@ -124,6 +124,7 @@ func (s *TiDBSource) GetCountAndCrc32(ctx context.Context, tableRange *splitter. beginTime := time.Now() table := s.tableDiffs[tableRange.GetTableIndex()] chunk := tableRange.GetChunk() + matchSource := getMatchSource(s.sourceTableMap, table) count, checksum, err := utils.GetCountAndCRC32Checksum(ctx, s.dbConn, matchSource.OriginSchema, matchSource.OriginTable, table.Info, chunk.Where, chunk.Args) diff --git a/tests/sync_diff_inspector/table_skip/data.sql b/tests/sync_diff_inspector/table_skip/data.sql index 7be211251..29943d49a 100644 --- a/tests/sync_diff_inspector/table_skip/data.sql +++ b/tests/sync_diff_inspector/table_skip/data.sql @@ -1,5 +1,4 @@ -drop database if exists diff_test; -create database diff_test; +create database diff_test if not exists diff_test; create table diff_test.t0 (a int, b int, primary key(a)); create table diff_test.t1 (a int, b int, primary key(a)); insert into diff_test.t0 values (1,1); From 8581b7b3b6838eba8e3947ce5e6267c30fe19828 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Mon, 30 Jan 2023 12:19:31 +0800 Subject: [PATCH 18/20] fix test --- tests/sync_diff_inspector/table_skip/data.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/sync_diff_inspector/table_skip/data.sql b/tests/sync_diff_inspector/table_skip/data.sql index 29943d49a..c8a36ccdb 100644 --- a/tests/sync_diff_inspector/table_skip/data.sql +++ b/tests/sync_diff_inspector/table_skip/data.sql @@ -1,4 +1,4 @@ -create database diff_test if not exists diff_test; +create database if not exists diff_test; create table diff_test.t0 (a int, b int, primary key(a)); create table diff_test.t1 (a int, b int, primary key(a)); insert into diff_test.t0 values (1,1); From a637154e08c655a52b52552ee5b6578273a0e7a1 Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Mon, 30 Jan 2023 13:18:45 +0800 Subject: [PATCH 19/20] fix test --- sync_diff_inspector/source/source.go | 7 ++-- .../table_skip/config_base.toml | 2 +- .../table_skip/config_router.toml | 17 +++------ tests/sync_diff_inspector/table_skip/data.sql | 10 ++--- tests/sync_diff_inspector/table_skip/run.sh | 38 +++++++++---------- 5 files changed, 34 insertions(+), 40 deletions(-) diff --git a/sync_diff_inspector/source/source.go b/sync_diff_inspector/source/source.go index a795d2a82..baa21e85d 100644 --- a/sync_diff_inspector/source/source.go +++ b/sync_diff_inspector/source/source.go @@ -188,10 +188,6 @@ func NewSources(ctx context.Context, cfg *config.Config) (downstream Source, ups } } - if len(tableDiffs) == 0 { - return nil, nil, errors.Errorf("no table need to be compared") - } - // Sort TableDiff is important! // because we compare table one by one. sort.Slice(tableDiffs, func(i, j int) bool { @@ -211,6 +207,9 @@ func NewSources(ctx context.Context, cfg *config.Config) (downstream Source, ups if err != nil { return nil, nil, errors.Annotate(err, "from upstream") } + if len(upstream.GetTables()) == 0 { + return nil, nil, errors.Errorf("no table need to be compared") + } downstream, err = buildSourceFromCfg(ctx, upstream.GetTables(), mysqlConnCount, bucketSpliterPool, cfg.SkipNonExistingTable, cfg.Task.TargetCheckTables, cfg.Task.TargetInstance) if err != nil { return nil, nil, errors.Annotate(err, "from downstream") diff --git a/tests/sync_diff_inspector/table_skip/config_base.toml b/tests/sync_diff_inspector/table_skip/config_base.toml index 1081c860d..ba3410073 100644 --- a/tests/sync_diff_inspector/table_skip/config_base.toml +++ b/tests/sync_diff_inspector/table_skip/config_base.toml @@ -46,4 +46,4 @@ skip-non-existing-table = true target-instance = "tidb" # tables need to check. - target-check-tables = ["diff_test.t*"] \ No newline at end of file + target-check-tables = ["skip_test.t*"] \ No newline at end of file diff --git a/tests/sync_diff_inspector/table_skip/config_router.toml b/tests/sync_diff_inspector/table_skip/config_router.toml index 9195e585b..2fa4ededf 100644 --- a/tests/sync_diff_inspector/table_skip/config_router.toml +++ b/tests/sync_diff_inspector/table_skip/config_router.toml @@ -28,7 +28,7 @@ skip-non-existing-table = true user = "root" password = "" - route-rules = ["rule2","rule3"] + route-rules = ["rule2"] [data-sources.tidb0] host = "127.0.0.1" @@ -38,22 +38,17 @@ skip-non-existing-table = true ########################### Routes ########################### [routes.rule1] -schema-pattern = "diff_test" # Matches the schema name of the data source. Supports the wildcards "*" and "?" +schema-pattern = "skip_test" # Matches the schema name of the data source. Supports the wildcards "*" and "?" table-pattern = "t[1-2]" # Matches the table name of the data source. Supports the wildcards "*" and "?" -target-schema = "diff_test" # The name of the schema in the target database +target-schema = "skip_test" # The name of the schema in the target database target-table = "t5" # The name of the target table [routes.rule2] -schema-pattern = "diff_test" +schema-pattern = "skip_test" table-pattern = "t0" -target-schema = "diff_test" +target-schema = "skip_test" target-table = "t5" -[routes.rule3] -schema-pattern = "diff_test" -table-pattern = "t4" -target-schema = "diff_test" -target-table = "t4" ######################### Task config ######################### [task] output-dir = "/tmp/tidb_tools_test/sync_diff_inspector/output" @@ -63,4 +58,4 @@ target-table = "t4" target-instance = "tidb0" # The tables of downstream databases to be compared. Each table needs to contain the schema name and the table name, separated by '.' - target-check-tables = ["diff_test.t5","diff_test.t4"] \ No newline at end of file + target-check-tables = ["skip_test.t5"] \ No newline at end of file diff --git a/tests/sync_diff_inspector/table_skip/data.sql b/tests/sync_diff_inspector/table_skip/data.sql index c8a36ccdb..34a2a7452 100644 --- a/tests/sync_diff_inspector/table_skip/data.sql +++ b/tests/sync_diff_inspector/table_skip/data.sql @@ -1,5 +1,5 @@ -create database if not exists diff_test; -create table diff_test.t0 (a int, b int, primary key(a)); -create table diff_test.t1 (a int, b int, primary key(a)); -insert into diff_test.t0 values (1,1); -insert into diff_test.t1 values (2,2); \ No newline at end of file +create database if not exists skip_test; +create table skip_test.t0 (a int, b int, primary key(a)); +create table skip_test.t1 (a int, b int, primary key(a)); +insert into skip_test.t0 values (1,1); +insert into skip_test.t1 values (2,2); \ No newline at end of file diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index a7e8c589b..9a217b89e 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -21,35 +21,35 @@ check_contains "check pass!!!" $OUT_DIR/sync_diff.log rm -rf $OUT_DIR/* echo "make some tables exist only upstream or downstream" -mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "create table diff_test.t2 (a int, b int, primary key(a));" -mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t2 values (3,3);" -mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t3 (a int, b int, primary key(a));" -mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t3 values (1,1);" +mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "create table skip_test.t2 (a int, b int, primary key(a));" +mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into skip_test.t2 values (3,3);" +mysql -uroot -h 127.0.0.1 -P 4000 -e "create table skip_test.t3 (a int, b int, primary key(a));" +mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into skip_test.t3 values (1,1);" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check pass" $OUT_DIR/sync_diff.log -check_contains "Comparing the table data of \`\`diff_test\`.\`t2\`\` ...skipped" $OUT_DIR/table_skip_diff.output -check_contains "Comparing the table data of \`\`diff_test\`.\`t3\`\` ...skipped" $OUT_DIR/table_skip_diff.output -check_contains "The data of \`diff_test\`.\`t2\` does not exist in downstream database" $OUT_DIR/table_skip_diff.output -check_contains "The data of \`diff_test\`.\`t3\` does not exist in upstream database" $OUT_DIR/table_skip_diff.output +check_contains "Comparing the table data of \`\`skip_test\`.\`t2\`\` ...skipped" $OUT_DIR/table_skip_diff.output +check_contains "Comparing the table data of \`\`skip_test\`.\`t3\`\` ...skipped" $OUT_DIR/table_skip_diff.output +check_contains "The data of \`skip_test\`.\`t2\` does not exist in downstream database" $OUT_DIR/table_skip_diff.output +check_contains "The data of \`skip_test\`.\`t3\` does not exist in upstream database" $OUT_DIR/table_skip_diff.output check_contains "| TABLE | RESULT | STRUCTURE EQUALITY | DATA DIFF ROWS | UPCOUNT | DOWNCOUNT |" $OUT_DIR/summary.txt -check_contains "| \`diff_test\`.\`t2\` | skipped | false | +1/-0 | 1 | 0 |" $OUT_DIR/summary.txt -check_contains "| \`diff_test\`.\`t3\` | skipped | false | +0/-1 | 0 | 1 |" $OUT_DIR/summary.txt +check_contains "| \`skip_test\`.\`t2\` | skipped | false | +1/-0 | 1 | 0 |" $OUT_DIR/summary.txt +check_contains "| \`skip_test\`.\`t3\` | skipped | false | +0/-1 | 0 | 1 |" $OUT_DIR/summary.txt rm -rf $OUT_DIR/* echo "make some table data not equal" -mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into diff_test.t1 values (4,4);" +mysql -uroot -h 127.0.0.1 -P 4000 -e "insert into skip_test.t1 values (4,4);" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log -check_contains "| \`diff_test\`.\`t1\` | succeed | true | +0/-1 | 1 | 2 |" $OUT_DIR/summary.txt +check_contains "| \`skip_test\`.\`t1\` | succeed | true | +0/-1 | 1 | 2 |" $OUT_DIR/summary.txt rm -rf $OUT_DIR/* echo "make some table structure not equal" -mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "create table diff_test.t4 (a int, b int, c int,primary key(a));" -mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into diff_test.t4 values (1,1,1);" -mysql -uroot -h 127.0.0.1 -P 4000 -e "create table diff_test.t4 (a int, b int, primary key(a));" +mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "create table skip_test.t4 (a int, b int, c int,primary key(a));" +mysql -uroot -h ${MYSQL_HOST} -P ${MYSQL_PORT} -e "insert into skip_test.t4 values (1,1,1);" +mysql -uroot -h 127.0.0.1 -P 4000 -e "create table skip_test.t4 (a int, b int, primary key(a));" sync_diff_inspector --config=./config.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log -check_contains "| \`diff_test\`.\`t4\` | succeed | false | +0/-0 | 0 | 0 |" $OUT_DIR/summary.txt +check_contains "| \`skip_test\`.\`t4\` | succeed | false | +0/-0 | 0 | 0 |" $OUT_DIR/summary.txt check_contains "A total of 5 tables have been compared, 1 tables finished, 2 tables failed, 2 tables skipped" $OUT_DIR/table_skip_diff.output cat $OUT_DIR/summary.txt rm -rf $OUT_DIR/* @@ -57,9 +57,9 @@ rm -rf $OUT_DIR/* echo "test router case" sync_diff_inspector --config=./config_router.toml > $OUT_DIR/table_skip_diff.output || true check_contains "check failed" $OUT_DIR/sync_diff.log -check_contains "| \`diff_test\`.\`t5\` | skipped | false | +3/-0 | 3 | 0 |" $OUT_DIR/summary.txt -check_contains "The data of \`diff_test\`.\`t5\` does not exist in downstream database" $OUT_DIR/table_skip_diff.output -check_contains "A total of 2 tables have been compared, 0 tables finished, 1 tables failed, 1 tables skipped" $OUT_DIR/table_skip_diff.output +check_contains "| \`skip_test\`.\`t5\` | skipped | false | +3/-0 | 3 | 0 |" $OUT_DIR/summary.txt +check_contains "The data of \`skip_test\`.\`t5\` does not exist in downstream database" $OUT_DIR/table_skip_diff.output +check_contains "A total of 1 tables have been compared, 0 tables finished, 0 tables failed, 1 tables skipped" $OUT_DIR/table_skip_diff.output rm -rf $OUT_DIR/* echo "table_skip test passed" \ No newline at end of file From 8d7c34361c98e26563a071f394a701df9b4ca28c Mon Sep 17 00:00:00 2001 From: liumengya94 <1827915285@qq.com> Date: Mon, 30 Jan 2023 13:31:36 +0800 Subject: [PATCH 20/20] fix test --- tests/sync_diff_inspector/table_skip/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/sync_diff_inspector/table_skip/run.sh b/tests/sync_diff_inspector/table_skip/run.sh index 9a217b89e..f0ed04cef 100644 --- a/tests/sync_diff_inspector/table_skip/run.sh +++ b/tests/sync_diff_inspector/table_skip/run.sh @@ -56,7 +56,7 @@ rm -rf $OUT_DIR/* echo "test router case" sync_diff_inspector --config=./config_router.toml > $OUT_DIR/table_skip_diff.output || true -check_contains "check failed" $OUT_DIR/sync_diff.log +check_contains "check pass" $OUT_DIR/sync_diff.log check_contains "| \`skip_test\`.\`t5\` | skipped | false | +3/-0 | 3 | 0 |" $OUT_DIR/summary.txt check_contains "The data of \`skip_test\`.\`t5\` does not exist in downstream database" $OUT_DIR/table_skip_diff.output check_contains "A total of 1 tables have been compared, 0 tables finished, 0 tables failed, 1 tables skipped" $OUT_DIR/table_skip_diff.output