diff --git a/dm/pkg/schema/tracker.go b/dm/pkg/schema/tracker.go index 1ee9d1015a4..de696115e71 100644 --- a/dm/pkg/schema/tracker.go +++ b/dm/pkg/schema/tracker.go @@ -397,13 +397,13 @@ func (tr *Tracker) GetDownStreamTableInfo(tctx *tcontext.Context, tableID string dti, ok := tr.dsTracker.tableInfos[tableID] if !ok { tctx.Logger.Info("Downstream schema tracker init. ", zap.String("tableID", tableID)) - ti, err := tr.getTableInfoByCreateStmt(tctx, tableID) + downstreamTI, err := tr.getTableInfoByCreateStmt(tctx, tableID) if err != nil { tctx.Logger.Error("Init dowstream schema info error. ", zap.String("tableID", tableID), zap.Error(err)) return nil, err } - dti = GetDownStreamTI(ti, originTi) + dti = GetDownStreamTI(downstreamTI, originTi) tr.dsTracker.tableInfos[tableID] = dti } return dti, nil diff --git a/dm/syncer/causality.go b/dm/syncer/causality.go index ab5eb8e8699..72b7546797b 100644 --- a/dm/syncer/causality.go +++ b/dm/syncer/causality.go @@ -20,6 +20,7 @@ import ( "go.uber.org/zap" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/syncer/metrics" ) @@ -79,7 +80,7 @@ func (c *causality) run() { c.relation.gc(j.flushSeq) continue default: - keys := j.dml.identifyKeys(c.sessCtx) + keys := j.dml.CausalityKeys() // detectConflict before add if c.detectConflict(keys) { @@ -87,8 +88,8 @@ func (c *causality) run() { c.outCh <- newConflictJob(c.workerCount) c.relation.clear() } - j.dml.key = c.add(keys) - c.logger.Debug("key for keys", zap.String("key", j.dml.key), zap.Strings("keys", keys)) + j.dmlQueueKey = c.add(keys) + c.logger.Debug("key for keys", zap.String("key", j.dmlQueueKey), zap.Strings("keys", keys)) } metrics.ConflictDetectDurationHistogram.WithLabelValues(c.task, c.source).Observe(time.Since(startTime).Seconds()) diff --git a/dm/syncer/causality_test.go b/dm/syncer/causality_test.go index 1109b8c1b80..b3acb6d0810 100644 --- a/dm/syncer/causality_test.go +++ b/dm/syncer/causality_test.go @@ -15,21 +15,19 @@ package syncer import ( "math" + "testing" "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb-tools/pkg/filter" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/types" - "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" + cdcmodel "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/binlog" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" - "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) func (s *testSyncerSuite) TestDetectConflict(c *C) { @@ -65,26 +63,11 @@ func (s *testSyncerSuite) TestDetectConflict(c *C) { c.Assert(ca.relation.len(), Equals, 0) } -func (s *testSyncerSuite) TestCasuality(c *C) { - p := parser.New() - se := mock.NewContext() +func TestCausality(t *testing.T) { + t.Parallel() + schemaStr := "create table tb(a int primary key, b int unique);" - ti, err := createTableInfo(p, se, int64(0), schemaStr) - c.Assert(err, IsNil) - tiIndex := &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti.Columns[0].Name, - Offset: ti.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi := schema.GetDownStreamTI(ti, ti) - c.Assert(downTi, NotNil) + ti := mockTableInfo(t, schemaStr) jobCh := make(chan *job, 10) syncer := &Syncer{ @@ -100,124 +83,44 @@ func (s *testSyncerSuite) TestCasuality(c *C) { } causalityCh := causalityWrap(jobCh, syncer) testCases := []struct { - op opType - oldVals []interface{} - vals []interface{} + preVals []interface{} + postVals []interface{} }{ { - op: insert, - vals: []interface{}{1, 2}, + postVals: []interface{}{1, 2}, }, { - op: insert, - vals: []interface{}{2, 3}, + postVals: []interface{}{2, 3}, }, { - op: update, - oldVals: []interface{}{2, 3}, - vals: []interface{}{3, 4}, + preVals: []interface{}{2, 3}, + postVals: []interface{}{3, 4}, }, { - op: del, - vals: []interface{}{1, 2}, + preVals: []interface{}{1, 2}, }, { - op: insert, - vals: []interface{}{1, 3}, + postVals: []interface{}{1, 3}, }, } - results := []opType{insert, insert, update, del, conflict, insert} - table := &filter.Table{Schema: "test", Name: "t1"} + results := []opType{dml, dml, dml, dml, conflict, dml} + table := &cdcmodel.TableName{Schema: "test", Table: "t1"} location := binlog.NewLocation("") ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} for _, tc := range testCases { - job := newDMLJob(tc.op, table, table, newDML(tc.op, false, "", table, tc.oldVals, tc.vals, tc.oldVals, tc.vals, ti.Columns, ti, tiIndex, downTi), ec) + change := sqlmodel.NewRowChange(table, nil, tc.preVals, tc.postVals, ti, nil, nil) + job := newDMLJob(change, ec) jobCh <- job } - c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { + require.Eventually(t, func() bool { return len(causalityCh) == len(results) - }), IsTrue) + }, 3*time.Second, 100*time.Millisecond) for _, op := range results { job := <-causalityCh - c.Assert(job.tp, Equals, op) - } -} - -func (s *testSyncerSuite) TestCasualityWithPrefixIndex(c *C) { - p := parser.New() - se := mock.NewContext() - schemaStr := "create table t (c1 text, c2 int unique, unique key c1(c1(3)));" - ti, err := createTableInfo(p, se, int64(0), schemaStr) - c.Assert(err, IsNil) - downTi := schema.GetDownStreamTI(ti, ti) - c.Assert(downTi, NotNil) - c.Assert(len(downTi.AvailableUKIndexList) == 2, IsTrue) - tiIndex := downTi.AvailableUKIndexList[0] - - jobCh := make(chan *job, 10) - syncer := &Syncer{ - cfg: &config.SubTaskConfig{ - SyncerConfig: config.SyncerConfig{ - QueueSize: 1024, - }, - Name: "task", - SourceID: "source", - }, - tctx: tcontext.Background().WithLogger(log.L()), - sessCtx: utils.NewSessionCtx(map[string]string{"time_zone": "UTC"}), - } - causalityCh := causalityWrap(jobCh, syncer) - testCases := []struct { - op opType - oldVals []interface{} - vals []interface{} - }{ - { - op: insert, - vals: []interface{}{"1234", 1}, - }, - { - op: insert, - vals: []interface{}{"2345", 2}, - }, - { - op: update, - oldVals: []interface{}{"2345", 2}, - vals: []interface{}{"2345", 3}, - }, - { - op: del, - vals: []interface{}{"1234", 1}, - }, - { - op: insert, - vals: []interface{}{"2345", 1}, - }, - } - results := []opType{insert, insert, update, del, conflict, insert} - resultKeys := []string{"123.c1.", "234.c1.", "234.c1.", "123.c1.", "conflict", "234.c1."} - table := &filter.Table{Schema: "test", Name: "t1"} - location := binlog.NewLocation("") - ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} - - for _, tc := range testCases { - job := newDMLJob(tc.op, table, table, newDML(tc.op, false, "", table, tc.oldVals, tc.vals, tc.oldVals, tc.vals, ti.Columns, ti, tiIndex, downTi), ec) - jobCh <- job - } - - c.Assert(utils.WaitSomething(30, 100*time.Millisecond, func() bool { - return len(causalityCh) == len(results) - }), IsTrue) - - for i, op := range results { - job := <-causalityCh - if job.tp != conflict { - c.Assert(job.dml.key, Equals, resultKeys[i]) - } - c.Assert(job.tp, Equals, op) + require.Equal(t, op, job.tp) } } diff --git a/dm/syncer/checkpoint_flush_worker.go b/dm/syncer/checkpoint_flush_worker.go index e781c4b49fd..6d7f12f3aff 100644 --- a/dm/syncer/checkpoint_flush_worker.go +++ b/dm/syncer/checkpoint_flush_worker.go @@ -37,11 +37,11 @@ type checkpointFlushTask struct { } type checkpointFlushWorker struct { - input chan *checkpointFlushTask - cp CheckPoint - execError *atomic.Error - afterFlushFn func(task *checkpointFlushTask) error - addCountFunc func(bool, string, opType, int64, *filter.Table) + input chan *checkpointFlushTask + cp CheckPoint + execError *atomic.Error + afterFlushFn func(task *checkpointFlushTask) error + updateJobMetricsFn func(bool, string, *job) } // Add add a new flush checkpoint job. @@ -60,7 +60,7 @@ func (w *checkpointFlushWorker) Run(ctx *tcontext.Context) { if isAsyncFlush { task.asyncflushJob.flushWg.Wait() - w.addCountFunc(true, adminQueueName, task.asyncflushJob.tp, 1, task.asyncflushJob.targetTable) + w.updateJobMetricsFn(true, adminQueueName, task.asyncflushJob) ctx.L().Info("async flush checkpoint snapshot job has been processed by dml worker, about to flush checkpoint snapshot", zap.Int64("job sequence", task.asyncflushJob.flushSeq), zap.Int("snapshot_id", task.snapshotInfo.id)) } else { ctx.L().Info("about to sync flush checkpoint snapshot", zap.Int("snapshot_id", task.snapshotInfo.id)) diff --git a/dm/syncer/compactor.go b/dm/syncer/compactor.go index 62c907f037d..65800a7a8f3 100644 --- a/dm/syncer/compactor.go +++ b/dm/syncer/compactor.go @@ -19,11 +19,11 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb-tools/pkg/filter" "go.uber.org/zap" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/syncer/metrics" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) // compactor compacts multiple statements into one statement. @@ -38,9 +38,9 @@ type compactor struct { buffer []*job // for metrics - task string - source string - addCountFunc func(bool, string, opType, int64, *filter.Table) + task string + source string + updateJobMetricsFn func(bool, string, *job) } // compactorWrap creates and runs a compactor instance. @@ -49,15 +49,15 @@ func compactorWrap(inCh chan *job, syncer *Syncer) chan *job { // TODO: implement ping-pong buffer. bufferSize := syncer.cfg.QueueSize * syncer.cfg.WorkerCount / 4 compactor := &compactor{ - inCh: inCh, - outCh: make(chan *job, bufferSize), - bufferSize: bufferSize, - logger: syncer.tctx.Logger.WithFields(zap.String("component", "compactor")), - keyMap: make(map[string]map[string]int), - buffer: make([]*job, 0, bufferSize), - task: syncer.cfg.Name, - source: syncer.cfg.SourceID, - addCountFunc: syncer.addCount, + inCh: inCh, + outCh: make(chan *job, bufferSize), + bufferSize: bufferSize, + logger: syncer.tctx.Logger.WithFields(zap.String("component", "compactor")), + keyMap: make(map[string]map[string]int), + buffer: make([]*job, 0, bufferSize), + task: syncer.cfg.Name, + source: syncer.cfg.SourceID, + updateJobMetricsFn: syncer.updateJobMetrics, } go func() { compactor.run() @@ -89,23 +89,21 @@ func (c *compactor) run() { // set safeMode when receive first job if len(c.buffer) == 0 { - c.safeMode = j.dml.safeMode + c.safeMode = j.safeMode } // if dml has no PK/NOT NULL UK, do not compact it. - if j.dml.identifyColumns() == nil { + if !j.dml.HasNotNullUniqueIdx() { c.buffer = append(c.buffer, j) continue } // if update job update its identify keys, turn it into delete + insert - if j.dml.op == update && j.dml.updateIdentify() { - delDML, insertDML := updateToDelAndInsert(j.dml) + if j.dml.IsIdentityUpdated() { + delDML, insertDML := j.dml.SplitUpdate() delJob := j.clone() - delJob.tp = del delJob.dml = delDML insertJob := j.clone() - insertJob.tp = insert insertJob.dml = insertDML c.compactJob(delJob) @@ -142,7 +140,7 @@ func (c *compactor) flushBuffer() { if j != nil { // set safemode for all jobs by first job in buffer. // or safemode for insert(delete + insert = insert with safemode) - j.dml.safeMode = c.safeMode || j.dml.safeMode + j.safeMode = c.safeMode || j.safeMode c.outCh <- j } } @@ -162,7 +160,7 @@ func (c *compactor) flushBuffer() { // DELETE + UPDATE => X _| // . func (c *compactor) compactJob(j *job) { - tableName := j.dml.targetTableID + tableName := j.dml.TargetTableID() tableKeyMap, ok := c.keyMap[tableName] if !ok { // do not alloc a large buffersize, otherwise if the downstream latency is low @@ -171,7 +169,7 @@ func (c *compactor) compactJob(j *job) { tableKeyMap = c.keyMap[tableName] } - key := j.dml.identifyKey() + key := j.dml.IdentityKey() failpoint.Inject("DownstreamIdentifyKeyCheckInCompact", func(v failpoint.Value) { value, err := strconv.Atoi(key) @@ -192,28 +190,24 @@ func (c *compactor) compactJob(j *job) { prevJob := c.buffer[prevPos] c.logger.Debug("start to compact", zap.Stringer("previous dml", prevJob.dml), zap.Stringer("current dml", j.dml)) - switch j.tp { - case update: - if prevJob.tp == insert { - // INSERT + UPDATE => INSERT - j.tp = insert - j.dml.oldValues = nil - j.dml.originOldValues = nil - j.dml.op = insert + // adjust safemode + skipReduce := false + switch j.dml.Type() { + case sqlmodel.RowChangeUpdate: + if prevJob.dml.Type() == sqlmodel.RowChangeInsert { // DELETE + INSERT + UPDATE => INSERT with safemode - j.dml.safeMode = prevJob.dml.safeMode - } else if prevJob.tp == update { - // UPDATE + UPDATE => UPDATE - j.dml.oldValues = prevJob.dml.oldValues - j.dml.originOldValues = prevJob.dml.originOldValues + j.safeMode = prevJob.safeMode } - case insert: - if prevJob.tp == del { + case sqlmodel.RowChangeInsert: + if prevJob.dml.Type() == sqlmodel.RowChangeDelete { // DELETE + INSERT => INSERT with safemode - j.dml.safeMode = true + j.safeMode = true + skipReduce = true } - case del: - // do nothing because anything + DELETE => DELETE + } + + if !skipReduce { + j.dml.Reduce(prevJob.dml) } // mark previous job as compacted(nil), add new job @@ -221,5 +215,5 @@ func (c *compactor) compactJob(j *job) { tableKeyMap[key] = len(c.buffer) c.buffer = append(c.buffer, j) c.logger.Debug("finish to compact", zap.Stringer("dml", j.dml)) - c.addCountFunc(true, adminQueueName, compact, 1, prevJob.targetTable) + c.updateJobMetricsFn(true, adminQueueName, newCompactJob(prevJob.targetTable)) } diff --git a/dm/syncer/compactor_test.go b/dm/syncer/compactor_test.go index 506f9581a7f..9118766ab1e 100644 --- a/dm/syncer/compactor_test.go +++ b/dm/syncer/compactor_test.go @@ -20,31 +20,29 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" - filter "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/util/mock" + cdcmodel "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/binlog" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" - "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) // mockExecute mock a kv store. -func mockExecute(kv map[interface{}][]interface{}, dmls []*DML) map[interface{}][]interface{} { +func mockExecute(kv map[interface{}][]interface{}, dmls []*sqlmodel.RowChange) map[interface{}][]interface{} { for _, dml := range dmls { - switch dml.op { - case insert: - kv[dml.values[0]] = dml.values - case update: - delete(kv, dml.oldValues[0]) - kv[dml.values[0]] = dml.values - case del: - delete(kv, dml.values[0]) + switch dml.Type() { + case sqlmodel.RowChangeInsert: + kv[dml.GetPostValues()[0]] = dml.GetPostValues() + case sqlmodel.RowChangeUpdate: + delete(kv, dml.GetPreValues()[0]) + kv[dml.GetPostValues()[0]] = dml.GetPostValues() + case sqlmodel.RowChangeDelete: + delete(kv, dml.GetPreValues()[0]) } } @@ -62,40 +60,25 @@ func randString(n int) string { func (s *testSyncerSuite) TestCompactJob(c *C) { compactor := &compactor{ - bufferSize: 10000, - logger: log.L(), - keyMap: make(map[string]map[string]int), - buffer: make([]*job, 0, 10000), - addCountFunc: func(b bool, s string, ot opType, i int64, t *filter.Table) {}, + bufferSize: 10000, + logger: log.L(), + keyMap: make(map[string]map[string]int), + buffer: make([]*job, 0, 10000), + updateJobMetricsFn: func(bool, string, *job) {}, } location := binlog.NewLocation("") ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} p := parser.New() se := mock.NewContext() - targetTableID := "`test`.`tb`" - sourceTable := &filter.Table{Schema: "test", Name: "tb1"} - targetTable := &filter.Table{Schema: "test", Name: "tb"} + sourceTable := &cdcmodel.TableName{Schema: "test", Table: "tb1"} + targetTable := &cdcmodel.TableName{Schema: "test", Table: "tb"} schemaStr := "create table test.tb(id int primary key, col1 int, name varchar(24))" ti, err := createTableInfo(p, se, 0, schemaStr) c.Assert(err, IsNil) - tiIndex := &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti.Columns[0].Name, - Offset: ti.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi := schema.GetDownStreamTI(ti, ti) - c.Assert(downTi, NotNil) - var dml *DML - var dmls []*DML + var dml *sqlmodel.RowChange + var dmls []*sqlmodel.RowChange dmlNum := 1000000 maxID := 1000 batch := 1000 @@ -111,7 +94,7 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { oldValues, ok := kv[newID] if !ok { // insert - dml = newDML(insert, false, targetTableID, sourceTable, nil, values, nil, values, ti.Columns, ti, tiIndex, downTi) + dml = sqlmodel.NewRowChange(sourceTable, targetTable, nil, values, ti, nil, nil) } else { if rand.Int()%2 > 0 { // update @@ -125,14 +108,14 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { } } } - dml = newDML(update, false, targetTableID, sourceTable, oldValues, values, oldValues, values, ti.Columns, ti, tiIndex, downTi) + dml = sqlmodel.NewRowChange(sourceTable, targetTable, oldValues, values, ti, nil, nil) } else { // delete - dml = newDML(del, false, targetTableID, sourceTable, nil, oldValues, nil, oldValues, ti.Columns, ti, tiIndex, downTi) + dml = sqlmodel.NewRowChange(sourceTable, targetTable, oldValues, nil, ti, nil, nil) } } - kv = mockExecute(kv, []*DML{dml}) + kv = mockExecute(kv, []*sqlmodel.RowChange{dml}) dmls = append(dmls, dml) } @@ -148,15 +131,14 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { kv = mockExecute(kv, dmls[i:end]) for _, dml := range dmls[i:end] { - j := newDMLJob(dml.op, sourceTable, targetTable, dml, ec) - if j.dml.op == update && j.dml.updateIdentify() { - delDML, insertDML := updateToDelAndInsert(j.dml) + j := newDMLJob(dml, ec) + // if update job update its identify keys, turn it into delete + insert + if j.dml.IsIdentityUpdated() { + delDML, insertDML := j.dml.SplitUpdate() delJob := j.clone() - delJob.tp = del delJob.dml = delDML insertJob := j.clone() - insertJob.tp = insert insertJob.dml = insertDML compactor.compactJob(delJob) @@ -173,12 +155,12 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { } for _, j := range compactor.buffer { if j != nil { - compactKV = mockExecute(compactKV, []*DML{j.dml}) + compactKV = mockExecute(compactKV, []*sqlmodel.RowChange{j.dml}) compactNumber++ c.Logf("after compact, dml: %s", j.dml.String()) } } - c.Logf("before compcat: %d, after compact: %d", noCompactNumber, compactNumber) + c.Logf("before compact: %d, after compact: %d", noCompactNumber, compactNumber) c.Assert(compactKV, DeepEquals, kv) compactor.keyMap = make(map[string]map[string]int) compactor.buffer = compactor.buffer[0:0] @@ -186,65 +168,101 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { } func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { - location := binlog.NewLocation("") - ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} p := parser.New() se := mock.NewContext() - targetTableID := "`test`.`tb`" - sourceTable := &filter.Table{Schema: "test", Name: "tb1"} - targetTable := &filter.Table{Schema: "test", Name: "tb"} + sourceTable := &cdcmodel.TableName{Schema: "test", Table: "tb"} schemaStr := "create table test.tb(id int primary key, col1 int, name varchar(24))" ti, err := createTableInfo(p, se, 0, schemaStr) c.Assert(err, IsNil) - tiIndex := &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti.Columns[0].Name, - Offset: ti.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi := schema.GetDownStreamTI(ti, ti) - c.Assert(downTi, NotNil) testCases := []struct { - input []*DML - output []*DML + input []*job + output []*job }{ // nolint:dupl { - input: []*DML{ - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(update, true, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), + input: []*job{ + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "b"}, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, nil, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ec, + ), }, - output: []*DML{ - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), - newDML(del, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), + output: []*job{ + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{3, 3, "c"}, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, nil, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ecWithSafeMode, + ), }, }, // nolint:dupl { - input: []*DML{ - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(update, false, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), - newDML(update, false, targetTableID, sourceTable, []interface{}{2, 2, "b"}, []interface{}{2, 2, "c"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "c"}, ti.Columns, ti, tiIndex, downTi), + input: []*job{ + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "b"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, nil, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "b"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, []interface{}{2, 2, "b"}, []interface{}{2, 2, "c"}, ti, nil, nil), + ec, + ), }, - output: []*DML{ - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "c"}, nil, []interface{}{2, 2, "c"}, ti.Columns, ti, tiIndex, downTi), + output: []*job{ + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{3, 3, "c"}, ti, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{1, 1, "a"}, ti, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable, nil, nil, []interface{}{2, 2, "c"}, ti, nil, nil), + ecWithSafeMode, + ), }, }, } @@ -269,8 +287,7 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { outCh := compactorWrap(inCh, syncer) for _, tc := range testCases { - for _, dml := range tc.input { - j := newDMLJob(dml.op, sourceTable, targetTable, dml, ec) + for _, j := range tc.input { inCh <- j } inCh <- newFlushJob(syncer.cfg.WorkerCount, 1) @@ -281,7 +298,7 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { for i := 0; i <= len(tc.output); i++ { j := <-outCh if i < len(tc.output) { - c.Assert(j.dml, DeepEquals, tc.output[i]) + c.Assert(j.String(), Equals, tc.output[i].String()) } else { c.Assert(j.tp, Equals, flush) } diff --git a/dm/syncer/ddl_test.go b/dm/syncer/ddl_test.go index 40342bc3b51..3b32ddc0f21 100644 --- a/dm/syncer/ddl_test.go +++ b/dm/syncer/ddl_test.go @@ -97,11 +97,11 @@ func (s *testDDLSuite) TestCommentQuote(c *C) { expectedSQL := "ALTER TABLE `schemadb`.`ep_edu_course_message_auto_reply` MODIFY COLUMN `answer` JSON COMMENT '回复的内容-格式为list,有两个字段:\"answerType\"://''发送客服消息类型:1-文本消息,2-图片,3-图文链接''; answer:回复内容'" tctx := tcontext.Background().WithLogger(log.With(zap.String("test", "TestCommentQuote"))) - ec := &eventContext{ + testEC := &eventContext{ tctx: tctx, } qec := &queryEventContext{ - eventContext: ec, + eventContext: testEC, ddlSchema: "schemadb", originSQL: sql, p: parser.New(), @@ -228,14 +228,14 @@ func (s *testDDLSuite) TestResolveDDLSQL(c *C) { }) c.Assert(err, IsNil) - ec := &eventContext{ + testEC := &eventContext{ tctx: tctx, } statusVars := []byte{4, 0, 0, 0, 0, 46, 0} syncer.idAndCollationMap = map[int]string{46: "utf8mb4_bin"} for i, sql := range sqls { qec := &queryEventContext{ - eventContext: ec, + eventContext: testEC, ddlSchema: "test", originSQL: sql, appliedDDLs: make([]string, 0), @@ -443,7 +443,7 @@ func (s *testDDLSuite) TestResolveOnlineDDL(c *C) { tctx := tcontext.Background().WithLogger(log.With(zap.String("test", "TestResolveOnlineDDL"))) p := parser.New() - ec := &eventContext{tctx: tctx} + testEC := &eventContext{tctx: tctx} cluster, err := conn.NewCluster() c.Assert(err, IsNil) c.Assert(cluster.Start(), IsNil) @@ -463,7 +463,7 @@ func (s *testDDLSuite) TestResolveOnlineDDL(c *C) { c.Assert(plugin.Clear(tctx), IsNil) c.Assert(syncer.genRouter(), IsNil) qec = &queryEventContext{ - eventContext: ec, + eventContext: testEC, ddlSchema: "test", appliedDDLs: make([]string, 0), p: p, diff --git a/dm/syncer/dml.go b/dm/syncer/dml.go index e5f5e8acc8f..7bf8c16a3c1 100644 --- a/dm/syncer/dml.go +++ b/dm/syncer/dml.go @@ -15,67 +15,32 @@ package syncer import ( "encoding/binary" - "fmt" - "strconv" - "strings" - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/tablecodec" "github.com/shopspring/decimal" "go.uber.org/zap" + cdcmodel "github.com/pingcap/tiflow/cdc/model" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" - "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) -// this type is used to generate DML SQL, opType is used to mark type in binlog. -type dmlOpType int64 - -const ( - insertDML = dmlOpType(insert) - updateDML = dmlOpType(update) - deleteDML = dmlOpType(del) - insertOnDuplicateDML dmlOpType = iota + 1 - replaceDML -) - -func (op dmlOpType) String() (str string) { - switch op { - case insertDML: - return "insert" - case updateDML: - return "update" - case deleteDML: - return "delete" - case insertOnDuplicateDML: - return "insert on duplicate update" - case replaceDML: - return "replace" - } - return -} - -// genDMLParam stores pruned columns, data as well as the original columns, data, index. +// genDMLParam stores original data and table structure. type genDMLParam struct { - targetTableID string // as a key in map like `schema`.`table` - sourceTable *filter.Table // origin table - safeMode bool // only used in update - data [][]interface{} // pruned data - originalData [][]interface{} // all data - columns []*model.ColumnInfo // pruned columns - sourceTableInfo *model.TableInfo // all table info - extendData [][]interface{} // all data include extend data + sourceTable *filter.Table // origin table + targetTable *filter.Table + safeMode bool // only used in update + originalData [][]interface{} // all data + sourceTableInfo *model.TableInfo // all table info + extendData [][]interface{} // all data include extend data } // extractValueFromData adjust the values obtained from go-mysql so that @@ -117,15 +82,13 @@ func extractValueFromData(data []interface{}, columns []*model.ColumnInfo, sourc return value } -func (s *Syncer) genAndFilterInsertDMLs(tctx *tcontext.Context, param *genDMLParam, filterExprs []expression.Expression) ([]*DML, error) { +func (s *Syncer) genAndFilterInsertDMLs(tctx *tcontext.Context, param *genDMLParam, filterExprs []expression.Expression) ([]*sqlmodel.RowChange, error) { var ( - tableID = param.targetTableID - dataSeq = param.data + tableID = utils.GenTableID(param.targetTable) originalDataSeq = param.originalData - columns = param.columns ti = param.sourceTableInfo extendData = param.extendData - dmls = make([]*DML, 0, len(dataSeq)) + dmls = make([]*sqlmodel.RowChange, 0, len(originalDataSeq)) ) // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) @@ -133,23 +96,18 @@ func (s *Syncer) genAndFilterInsertDMLs(tctx *tcontext.Context, param *genDMLPar if err != nil { return nil, err } - downstreamIndexColumns := downstreamTableInfo.AbsoluteUKIndexInfo if extendData != nil { originalDataSeq = extendData } RowLoop: - for dataIdx, data := range dataSeq { - if len(data) != len(columns) { - return nil, terror.ErrSyncerUnitDMLColumnNotMatch.Generate(len(columns), len(data)) + for dataIdx, data := range originalDataSeq { + if len(data) != len(ti.Columns) { + return nil, terror.ErrSyncerUnitDMLColumnNotMatch.Generate(len(ti.Columns), len(data)) } - value := extractValueFromData(data, columns, ti) - originalValue := value - if len(columns) != len(ti.Columns) { - originalValue = extractValueFromData(originalDataSeq[dataIdx], ti.Columns, ti) - } + originalValue := extractValueFromData(originalDataSeq[dataIdx], ti.Columns, ti) for _, expr := range filterExprs { skip, err := SkipDMLByExpression(s.sessCtx, originalValue, expr, ti.Columns) @@ -162,11 +120,17 @@ RowLoop: } } - if downstreamIndexColumns == nil { - downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, value) - } - - dmls = append(dmls, newDML(insert, param.safeMode, tableID, param.sourceTable, nil, value, nil, originalValue, columns, ti, downstreamIndexColumns, downstreamTableInfo)) + rowChange := sqlmodel.NewRowChange( + &cdcmodel.TableName{Schema: param.sourceTable.Schema, Table: param.sourceTable.Name}, + &cdcmodel.TableName{Schema: param.targetTable.Schema, Table: param.targetTable.Name}, + nil, + originalValue, + param.sourceTableInfo, + downstreamTableInfo.TableInfo, + s.sessCtx, + ) + rowChange.SetIdentifyInfo(downstreamTableInfo) + dmls = append(dmls, rowChange) } return dmls, nil @@ -177,15 +141,13 @@ func (s *Syncer) genAndFilterUpdateDMLs( param *genDMLParam, oldValueFilters []expression.Expression, newValueFilters []expression.Expression, -) ([]*DML, error) { +) ([]*sqlmodel.RowChange, error) { var ( - tableID = param.targetTableID - data = param.data + tableID = utils.GenTableID(param.targetTable) originalData = param.originalData - columns = param.columns ti = param.sourceTableInfo extendData = param.extendData - dmls = make([]*DML, 0, len(data)/2) + dmls = make([]*sqlmodel.RowChange, 0, len(originalData)/2) ) // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) @@ -193,38 +155,26 @@ func (s *Syncer) genAndFilterUpdateDMLs( if err != nil { return nil, err } - downstreamIndexColumns := downstreamTableInfo.AbsoluteUKIndexInfo if extendData != nil { originalData = extendData } RowLoop: - for i := 0; i < len(data); i += 2 { - oldData := data[i] - changedData := data[i+1] + for i := 0; i < len(originalData); i += 2 { oriOldData := originalData[i] oriChangedData := originalData[i+1] - if len(oldData) != len(changedData) { - return nil, terror.ErrSyncerUnitDMLOldNewValueMismatch.Generate(len(oldData), len(changedData)) + if len(oriOldData) != len(oriChangedData) { + return nil, terror.ErrSyncerUnitDMLOldNewValueMismatch.Generate(len(oriOldData), len(oriChangedData)) } - if len(oldData) != len(columns) { - return nil, terror.ErrSyncerUnitDMLColumnNotMatch.Generate(len(columns), len(oldData)) + if len(oriOldData) != len(ti.Columns) { + return nil, terror.ErrSyncerUnitDMLColumnNotMatch.Generate(len(ti.Columns), len(oriOldData)) } - oldValues := extractValueFromData(oldData, columns, ti) - changedValues := extractValueFromData(changedData, columns, ti) - - var oriOldValues, oriChangedValues []interface{} - if len(columns) == len(ti.Columns) { - oriOldValues = oldValues - oriChangedValues = changedValues - } else { - oriOldValues = extractValueFromData(oriOldData, ti.Columns, ti) - oriChangedValues = extractValueFromData(oriChangedData, ti.Columns, ti) - } + oriOldValues := extractValueFromData(oriOldData, ti.Columns, ti) + oriChangedValues := extractValueFromData(oriChangedData, ti.Columns, ti) for j := range oldValueFilters { // AND logic @@ -244,23 +194,29 @@ RowLoop: } } - if downstreamIndexColumns == nil { - downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, oriOldValues) - } - - dmls = append(dmls, newDML(update, param.safeMode, param.targetTableID, param.sourceTable, oldValues, changedValues, oriOldValues, oriChangedValues, columns, ti, downstreamIndexColumns, downstreamTableInfo)) + rowChange := sqlmodel.NewRowChange( + &cdcmodel.TableName{Schema: param.sourceTable.Schema, Table: param.sourceTable.Name}, + &cdcmodel.TableName{Schema: param.targetTable.Schema, Table: param.targetTable.Name}, + oriOldValues, + oriChangedValues, + param.sourceTableInfo, + downstreamTableInfo.TableInfo, + s.sessCtx, + ) + rowChange.SetIdentifyInfo(downstreamTableInfo) + dmls = append(dmls, rowChange) } return dmls, nil } -func (s *Syncer) genAndFilterDeleteDMLs(tctx *tcontext.Context, param *genDMLParam, filterExprs []expression.Expression) ([]*DML, error) { +func (s *Syncer) genAndFilterDeleteDMLs(tctx *tcontext.Context, param *genDMLParam, filterExprs []expression.Expression) ([]*sqlmodel.RowChange, error) { var ( - tableID = param.targetTableID + tableID = utils.GenTableID(param.targetTable) dataSeq = param.originalData ti = param.sourceTableInfo extendData = param.extendData - dmls = make([]*DML, 0, len(dataSeq)) + dmls = make([]*sqlmodel.RowChange, 0, len(dataSeq)) ) // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) @@ -268,7 +224,6 @@ func (s *Syncer) genAndFilterDeleteDMLs(tctx *tcontext.Context, param *genDMLPar if err != nil { return nil, err } - downstreamIndexColumns := downstreamTableInfo.AbsoluteUKIndexInfo if extendData != nil { dataSeq = extendData @@ -293,11 +248,17 @@ RowLoop: } } - if downstreamIndexColumns == nil { - downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, value) - } - - dmls = append(dmls, newDML(del, false, param.targetTableID, param.sourceTable, nil, value, nil, value, ti.Columns, ti, downstreamIndexColumns, downstreamTableInfo)) + rowChange := sqlmodel.NewRowChange( + &cdcmodel.TableName{Schema: param.sourceTable.Schema, Table: param.sourceTable.Name}, + &cdcmodel.TableName{Schema: param.targetTable.Schema, Table: param.targetTable.Name}, + value, + nil, + param.sourceTableInfo, + downstreamTableInfo.TableInfo, + s.sessCtx, + ) + rowChange.SetIdentifyInfo(downstreamTableInfo) + dmls = append(dmls, rowChange) } return dmls, nil @@ -332,115 +293,6 @@ func castUnsigned(data interface{}, ft *types.FieldType) interface{} { return data } -func columnValue(value interface{}, ft *types.FieldType) string { - castValue := castUnsigned(value, ft) - - var data string - switch v := castValue.(type) { - case nil: - data = "null" - case bool: - if v { - data = "1" - } else { - data = "0" - } - case int: - data = strconv.FormatInt(int64(v), 10) - case int8: - data = strconv.FormatInt(int64(v), 10) - case int16: - data = strconv.FormatInt(int64(v), 10) - case int32: - data = strconv.FormatInt(int64(v), 10) - case int64: - data = strconv.FormatInt(v, 10) - case uint8: - data = strconv.FormatUint(uint64(v), 10) - case uint16: - data = strconv.FormatUint(uint64(v), 10) - case uint32: - data = strconv.FormatUint(uint64(v), 10) - case uint64: - data = strconv.FormatUint(v, 10) - case float32: - data = strconv.FormatFloat(float64(v), 'f', -1, 32) - case float64: - data = strconv.FormatFloat(v, 'f', -1, 64) - case string: - data = v - case []byte: - data = string(v) - default: - data = fmt.Sprintf("%v", v) - } - - return data -} - -func findFitIndex(ti *model.TableInfo) *model.IndexInfo { - for _, idx := range ti.Indices { - if idx.Primary { - return idx - } - } - - if pk := ti.GetPkColInfo(); pk != nil { - return &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: pk.Name, - Offset: pk.Offset, - Length: types.UnspecifiedLength, - }}, - } - } - - // second find not null unique key - fn := func(i int) bool { - return !mysql.HasNotNullFlag(ti.Columns[i].Flag) - } - - return getSpecifiedIndexColumn(ti, fn) -} - -func getSpecifiedIndexColumn(ti *model.TableInfo, fn func(i int) bool) *model.IndexInfo { - for _, indexCols := range ti.Indices { - if !indexCols.Unique { - continue - } - - findFitIndex := true - for _, col := range indexCols.Columns { - if fn(col.Offset) { - findFitIndex = false - break - } - } - - if findFitIndex { - return indexCols - } - } - - return nil -} - -func getColumnData(columns []*model.ColumnInfo, indexColumns *model.IndexInfo, data []interface{}) ([]*model.ColumnInfo, []interface{}) { - cols := make([]*model.ColumnInfo, 0, len(indexColumns.Columns)) - values := make([]interface{}, 0, len(indexColumns.Columns)) - for _, column := range indexColumns.Columns { - cols = append(cols, columns[column.Offset]) - values = append(values, data[column.Offset]) - } - - return cols, values -} - func (s *Syncer) mappingDML(table *filter.Table, ti *model.TableInfo, data [][]interface{}) ([][]interface{}, error) { if s.columnMapping == nil { return data, nil @@ -464,49 +316,6 @@ func (s *Syncer) mappingDML(table *filter.Table, ti *model.TableInfo, data [][]i return rows, nil } -// pruneGeneratedColumnDML filters columns list, data and index removing all -// generated column. because generated column is not support setting value -// directly in DML, we must remove generated column from DML, including column -// list and data list including generated columns. -func pruneGeneratedColumnDML(ti *model.TableInfo, data [][]interface{}) ([]*model.ColumnInfo, [][]interface{}, error) { - // search for generated columns. if none found, return everything as-is. - firstGeneratedColumnIndex := -1 - for i, c := range ti.Columns { - if c.IsGenerated() { - firstGeneratedColumnIndex = i - break - } - } - if firstGeneratedColumnIndex < 0 { - return ti.Columns, data, nil - } - - // remove generated columns from the list of columns - cols := make([]*model.ColumnInfo, 0, len(ti.Columns)) - cols = append(cols, ti.Columns[:firstGeneratedColumnIndex]...) - for _, c := range ti.Columns[(firstGeneratedColumnIndex + 1):] { - if !c.IsGenerated() { - cols = append(cols, c) - } - } - - // remove generated columns from the list of data. - rows := make([][]interface{}, 0, len(data)) - for _, row := range data { - if len(row) != len(ti.Columns) { - return nil, nil, terror.ErrSyncerUnitDMLPruneColumnMismatch.Generate(len(ti.Columns), len(data)) - } - value := make([]interface{}, 0, len(cols)) - for i := range row { - if !ti.Columns[i].IsGenerated() { - value = append(value, row[i]) - } - } - rows = append(rows, value) - } - return cols, rows, nil -} - // checkLogColumns returns error when not all rows in skipped is empty, which means the binlog doesn't contain all // columns. // TODO: don't return error when all skipped columns is non-PK. @@ -519,518 +328,20 @@ func checkLogColumns(skipped [][]int) error { return nil } -// DML stores param for DML. -type DML struct { - targetTableID string - sourceTable *filter.Table - op opType - oldValues []interface{} // only for update SQL - values []interface{} - columns []*model.ColumnInfo - sourceTableInfo *model.TableInfo - originOldValues []interface{} // only for update SQL - originValues []interface{} // use to gen key and `WHERE` - safeMode bool - key string // use to detect causality - pickedDownstreamIndexInfo *model.IndexInfo // pick an index from downstream which comes from pk/uk not null/uk value not null and is only used in genWhere - downstreamTableInfo *schema.DownstreamTableInfo // downstream table info -} - -// newDML creates DML. -func newDML(op opType, safeMode bool, targetTableID string, sourceTable *filter.Table, oldValues, values, originOldValues, originValues []interface{}, columns []*model.ColumnInfo, sourceTableInfo *model.TableInfo, pickedDownstreamIndexInfo *model.IndexInfo, downstreamTableInfo *schema.DownstreamTableInfo) *DML { - return &DML{ - op: op, - safeMode: safeMode, - targetTableID: targetTableID, - sourceTable: sourceTable, - oldValues: oldValues, - values: values, - columns: columns, - sourceTableInfo: sourceTableInfo, - originOldValues: originOldValues, - originValues: originValues, - pickedDownstreamIndexInfo: pickedDownstreamIndexInfo, - downstreamTableInfo: downstreamTableInfo, - } -} - -// String returns the DML's string. -func (dml *DML) String() string { - return fmt.Sprintf("[safemode: %t, targetTableID: %s, op: %s, columns: %v, oldValues: %v, values: %v]", dml.safeMode, dml.targetTableID, dml.op.String(), dml.columnNames(), dml.originOldValues, dml.originValues) -} - -// updateToDelAndInsert turns updateDML to delDML and insertDML. -func updateToDelAndInsert(updateDML *DML) (*DML, *DML) { - delDML := &DML{} - *delDML = *updateDML - delDML.op = del - // use oldValues of update as values of delete and reset oldValues - delDML.values = updateDML.oldValues - delDML.originValues = updateDML.originOldValues - delDML.oldValues = nil - delDML.originOldValues = nil - - insertDML := &DML{} - *insertDML = *updateDML - insertDML.op = insert - // reset oldValues - insertDML.oldValues = nil - insertDML.originOldValues = nil - - return delDML, insertDML -} - -// identifyColumns gets columns of unique not null index. -// This is used for compact. -func (dml *DML) identifyColumns() []string { - if defaultIndexColumns := dml.downstreamTableInfo.AbsoluteUKIndexInfo; defaultIndexColumns != nil { - columns := make([]string, 0, len(defaultIndexColumns.Columns)) - for _, column := range defaultIndexColumns.Columns { - columns = append(columns, column.Name.O) - } - return columns - } - return nil -} - -// identifyValues gets values of unique not null index. -// This is used for compact. -func (dml *DML) identifyValues() []interface{} { - if defaultIndexColumns := dml.downstreamTableInfo.AbsoluteUKIndexInfo; defaultIndexColumns != nil { - values := make([]interface{}, 0, len(defaultIndexColumns.Columns)) - for _, column := range defaultIndexColumns.Columns { - values = append(values, dml.values[column.Offset]) - } - return values - } - return nil -} - -// oldIdentifyValues gets old values of unique not null index. -// only for update SQL. -func (dml *DML) oldIdentifyValues() []interface{} { - if defaultIndexColumns := dml.downstreamTableInfo.AbsoluteUKIndexInfo; defaultIndexColumns != nil { - values := make([]interface{}, 0, len(defaultIndexColumns.Columns)) - for _, column := range defaultIndexColumns.Columns { - values = append(values, dml.oldValues[column.Offset]) - } - return values - } - return nil -} - -// identifyKey use identifyValues to gen key. -// This is used for compact. -// PK or (UK + NOT NULL). -func (dml *DML) identifyKey() string { - return genKey(dml.identifyValues()) -} - -// updateIdentify check whether a update sql update its identify values. -func (dml *DML) updateIdentify() bool { - if len(dml.oldValues) == 0 { - return false - } - - values := dml.identifyValues() - oldValues := dml.oldIdentifyValues() - - if len(values) != len(oldValues) { - return true - } - - for i := 0; i < len(values); i++ { - if values[i] != oldValues[i] { - return true - } - } - - return false -} - -// identifyKeys gens keys by unique not null value. -// This is used for causality. -// PK or (UK + NOT NULL) or (UK + NULL + NOT NULL VALUE). -func (dml *DML) identifyKeys(ctx sessionctx.Context) []string { - var keys []string - // for UPDATE statement - if dml.originOldValues != nil { - keys = append(keys, genMultipleKeys(ctx, dml.downstreamTableInfo, dml.sourceTableInfo, dml.originOldValues, dml.targetTableID)...) - } - - if dml.originValues != nil { - keys = append(keys, genMultipleKeys(ctx, dml.downstreamTableInfo, dml.sourceTableInfo, dml.originValues, dml.targetTableID)...) - } - return keys -} - -// columnNames return column names of DML. -func (dml *DML) columnNames() []string { - columnNames := make([]string, 0, len(dml.columns)) - for _, column := range dml.columns { - columnNames = append(columnNames, column.Name.O) - } - return columnNames -} - -// whereColumnsAndValues gets columns and values of unique column with not null value. -// This is used to generete where condition. -func (dml *DML) whereColumnsAndValues() ([]string, []interface{}) { - columns, values := dml.sourceTableInfo.Columns, dml.originValues - - if dml.op == update { - values = dml.originOldValues - } - - if dml.pickedDownstreamIndexInfo != nil { - columns, values = getColumnData(dml.sourceTableInfo.Columns, dml.pickedDownstreamIndexInfo, values) - } - - columnNames := make([]string, 0, len(columns)) - for _, column := range columns { - columnNames = append(columnNames, column.Name.O) - } - - failpoint.Inject("DownstreamTrackerWhereCheck", func() { - if dml.op == update { - log.L().Info("UpdateWhereColumnsCheck", zap.String("Columns", fmt.Sprintf("%v", columnNames))) - } else if dml.op == del { - log.L().Info("DeleteWhereColumnsCheck", zap.String("Columns", fmt.Sprintf("%v", columnNames))) - } - }) - - return columnNames, values -} - -// genKey gens key by values e.g. "a.1.b". -// This is used for compact. -func genKey(values []interface{}) string { - builder := new(strings.Builder) - for i, v := range values { - if i != 0 { - builder.WriteString(".") - } - fmt.Fprintf(builder, "%v", v) - } - - return builder.String() -} - -// genKeyList format keys. -func genKeyList(table string, columns []*model.ColumnInfo, dataSeq []interface{}) string { - var buf strings.Builder - for i, data := range dataSeq { - if data == nil { - log.L().Debug("ignore null value", zap.String("column", columns[i].Name.O), zap.String("table", table)) - continue // ignore `null` value. - } - // one column key looks like:`column_val.column_name.` - buf.WriteString(columnValue(data, &columns[i].FieldType)) - buf.WriteString(".") - buf.WriteString(columns[i].Name.O) - buf.WriteString(".") - } - if buf.Len() == 0 { - log.L().Debug("all value are nil, no key generated", zap.String("table", table)) - return "" // all values are `null`. - } - buf.WriteString(table) - return buf.String() -} - -// truncateIndexValues truncate prefix index from data. -func truncateIndexValues(ctx sessionctx.Context, ti *model.TableInfo, indexColumns *model.IndexInfo, tiColumns []*model.ColumnInfo, data []interface{}) []interface{} { - values := make([]interface{}, 0, len(indexColumns.Columns)) - datums, err := utils.AdjustBinaryProtocolForDatum(ctx, data, tiColumns) - if err != nil { - log.L().Warn("adjust binary protocol for datum error", zap.Error(err)) - return data - } - tablecodec.TruncateIndexValues(ti, indexColumns, datums) - for _, datum := range datums { - values = append(values, datum.GetValue()) - } - return values -} - -// genMultipleKeys gens keys with UNIQUE NOT NULL value. -// if not UNIQUE NOT NULL value, use table name instead. -func genMultipleKeys(ctx sessionctx.Context, downstreamTableInfo *schema.DownstreamTableInfo, ti *model.TableInfo, value []interface{}, table string) []string { - multipleKeys := make([]string, 0, len(downstreamTableInfo.AvailableUKIndexList)) - - for _, indexCols := range downstreamTableInfo.AvailableUKIndexList { - cols, vals := getColumnData(ti.Columns, indexCols, value) - // handle prefix index - truncVals := truncateIndexValues(ctx, ti, indexCols, cols, vals) - key := genKeyList(table, cols, truncVals) - if len(key) > 0 { // ignore `null` value. - multipleKeys = append(multipleKeys, key) - } else { - log.L().Debug("ignore empty key", zap.String("table", table)) - } - } - - if len(multipleKeys) == 0 { - // use table name as key if no key generated (no PK/UK), - // no concurrence for rows in the same table. - log.L().Debug("use table name as the key", zap.String("table", table)) - multipleKeys = append(multipleKeys, table) - } - - return multipleKeys -} - -// genWhere generates where condition. -func (dml *DML) genWhere(buf *strings.Builder) []interface{} { - whereColumns, whereValues := dml.whereColumnsAndValues() - - for i, col := range whereColumns { - if i != 0 { - buf.WriteString(" AND ") - } - buf.WriteString(dbutil.ColumnName(col)) - if whereValues[i] == nil { - buf.WriteString(" IS ?") - } else { - buf.WriteString(" = ?") - } - } - return whereValues -} - -// genSQL generates SQL for a DML. -func (dml *DML) genSQL() (sql []string, arg [][]interface{}) { - switch dml.op { - case insert: - return dml.genInsertSQL() - case del: - return dml.genDeleteSQL() - case update: - return dml.genUpdateSQL() - } - return -} - -// genUpdateSQL generates a `UPDATE` SQL with `WHERE`. -func (dml *DML) genUpdateSQL() ([]string, [][]interface{}) { - if dml.safeMode { - sqls, args := dml.genDeleteSQL() - insertSQLs, insertArgs := dml.genInsertSQL() - sqls = append(sqls, insertSQLs...) - args = append(args, insertArgs...) - return sqls, args - } - var buf strings.Builder - buf.Grow(2048) - buf.WriteString("UPDATE ") - buf.WriteString(dml.targetTableID) - buf.WriteString(" SET ") - - for i, column := range dml.columns { - if i == len(dml.columns)-1 { - fmt.Fprintf(&buf, "%s = ?", dbutil.ColumnName(column.Name.O)) - } else { - fmt.Fprintf(&buf, "%s = ?, ", dbutil.ColumnName(column.Name.O)) - } - } - - buf.WriteString(" WHERE ") - whereArgs := dml.genWhere(&buf) - buf.WriteString(" LIMIT 1") - - args := dml.values - args = append(args, whereArgs...) - return []string{buf.String()}, [][]interface{}{args} -} - -// genDeleteSQL generates a `DELETE FROM` SQL with `WHERE`. -func (dml *DML) genDeleteSQL() ([]string, [][]interface{}) { - var buf strings.Builder - buf.Grow(1024) - buf.WriteString("DELETE FROM ") - buf.WriteString(dml.targetTableID) - buf.WriteString(" WHERE ") - whereArgs := dml.genWhere(&buf) - buf.WriteString(" LIMIT 1") - - return []string{buf.String()}, [][]interface{}{whereArgs} -} - -// genInsertSQL generates a `INSERT`. -// if in safemode, generates a `REPLACE` statement. -func (dml *DML) genInsertSQL() ([]string, [][]interface{}) { - var buf strings.Builder - buf.Grow(1024) - if dml.safeMode { - buf.WriteString("REPLACE INTO ") - } else { - buf.WriteString("INSERT INTO ") - } - buf.WriteString(dml.targetTableID) - buf.WriteString(" (") - for i, column := range dml.columns { - buf.WriteString(dbutil.ColumnName(column.Name.O)) - if i != len(dml.columns)-1 { - buf.WriteByte(',') - } else { - buf.WriteByte(')') - } - } - buf.WriteString(" VALUES (") - - // placeholders - for i := range dml.columns { - if i != len(dml.columns)-1 { - buf.WriteString("?,") - } else { - buf.WriteString("?)") - } - } - return []string{buf.String()}, [][]interface{}{dml.values} -} - -// valuesHolder gens values holder like (?,?,?). -func valuesHolder(n int) string { - builder := new(strings.Builder) - builder.WriteByte('(') - for i := 0; i < n; i++ { - if i > 0 { - builder.WriteString(",") - } - builder.WriteString("?") - } - builder.WriteByte(')') - return builder.String() -} - -// genInsertSQLMultipleRows generates a `INSERT` with multiple rows like 'INSERT INTO tb(a,b) VALUES (1,1),(2,2)' -// if replace, generates a `REPLACE' with multiple rows like 'REPLACE INTO tb(a,b) VALUES (1,1),(2,2)' -// if onDuplicate, generates a `INSERT ON DUPLICATE KEY UPDATE` statement like 'INSERT INTO tb(a,b) VALUES (1,1),(2,2) ON DUPLICATE KEY UPDATE a=VALUES(a),b=VALUES(b)'. -func genInsertSQLMultipleRows(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) { - if len(dmls) == 0 { - return nil, nil - } - - var buf strings.Builder - buf.Grow(1024) - if op == replaceDML { - buf.WriteString("REPLACE INTO") - } else { - buf.WriteString("INSERT INTO") - } - buf.WriteString(" " + dmls[0].targetTableID + " (") - for i, column := range dmls[0].columns { - buf.WriteString(dbutil.ColumnName(column.Name.O)) - if i != len(dmls[0].columns)-1 { - buf.WriteByte(',') - } else { - buf.WriteByte(')') - } - } - buf.WriteString(" VALUES ") - - holder := valuesHolder(len(dmls[0].columns)) - for i := range dmls { - if i > 0 { - buf.WriteString(",") - } - buf.WriteString(holder) - } - - if op == insertOnDuplicateDML { - buf.WriteString(" ON DUPLICATE KEY UPDATE ") - for i, column := range dmls[0].columns { - col := dbutil.ColumnName(column.Name.O) - buf.WriteString(col + "=VALUES(" + col + ")") - if i != len(dmls[0].columns)-1 { - buf.WriteByte(',') - } - } - } - - args := make([]interface{}, 0, len(dmls)*len(dmls[0].columns)) - for _, dml := range dmls { - args = append(args, dml.values...) - } - return []string{buf.String()}, [][]interface{}{args} -} - -// genDeleteSQLMultipleRows generates delete statement with multiple rows like 'DELETE FROM tb WHERE (a,b) IN (1,1),(2,2)'. -func genDeleteSQLMultipleRows(dmls []*DML) ([]string, [][]interface{}) { - if len(dmls) == 0 { - return nil, nil - } - - var buf strings.Builder - buf.Grow(1024) - buf.WriteString("DELETE FROM ") - buf.WriteString(dmls[0].targetTableID) - buf.WriteString(" WHERE (") - - whereColumns, _ := dmls[0].whereColumnsAndValues() - for i, column := range whereColumns { - if i != len(whereColumns)-1 { - buf.WriteString(dbutil.ColumnName(column) + ",") - } else { - buf.WriteString(dbutil.ColumnName(column) + ")") - } - } - buf.WriteString(" IN (") - - holder := valuesHolder(len(whereColumns)) - args := make([]interface{}, 0, len(dmls)*len(dmls[0].columns)) - for i, dml := range dmls { - if i > 0 { - buf.WriteString(",") - } - buf.WriteString(holder) - _, whereValues := dml.whereColumnsAndValues() - // whereValues will have same length because we have checked it in genDMLsWithSameCols. - args = append(args, whereValues...) - } - buf.WriteString(")") - return []string{buf.String()}, [][]interface{}{args} -} - // genSQLMultipleRows generates multiple rows SQL with different dmlOpType. -func genSQLMultipleRows(op dmlOpType, dmls []*DML) (queries []string, args [][]interface{}) { +func genSQLMultipleRows(op sqlmodel.DMLType, dmls []*sqlmodel.RowChange) (queries string, args []interface{}) { if len(dmls) > 1 { - log.L().Debug("generate DMLs with multiple rows", zap.Stringer("op", op), zap.Stringer("original op", dmls[0].op), zap.Int("rows", len(dmls))) + log.L().Debug("generate DMLs with multiple rows", zap.Stringer("op", op), zap.Stringer("original op", dmls[0].Type()), zap.Int("rows", len(dmls))) } switch op { - case insertDML, replaceDML, insertOnDuplicateDML: - return genInsertSQLMultipleRows(op, dmls) - case deleteDML: - return genDeleteSQLMultipleRows(dmls) + case sqlmodel.DMLInsert, sqlmodel.DMLReplace, sqlmodel.DMLInsertOnDuplicateUpdate: + return sqlmodel.GenInsertSQL(op, dmls...) + case sqlmodel.DMLDelete: + return sqlmodel.GenDeleteSQL(dmls...) } return } -// sameColumns check whether two DMLs have same columns. -func sameColumns(lhs *DML, rhs *DML) bool { - var lhsCols, rhsCols []string - if lhs.op == del { - lhsCols, _ = lhs.whereColumnsAndValues() - rhsCols, _ = rhs.whereColumnsAndValues() - } else { - // if source table is same, columns will be same. - if lhs.sourceTable.Schema == rhs.sourceTable.Schema && lhs.sourceTable.Name == rhs.sourceTable.Name { - return true - } - lhsCols = lhs.columnNames() - rhsCols = rhs.columnNames() - } - if len(lhsCols) != len(rhsCols) { - return false - } - for i := 0; i < len(lhsCols); i++ { - if lhsCols[i] != rhsCols[i] { - return false - } - } - return true -} - // genDMLsWithSameCols group and gen dmls by same columns. // in optimistic shard mode, different upstream tables may have different columns. // e.g. @@ -1038,23 +349,23 @@ func sameColumns(lhs *DML, rhs *DML) bool { // insert into tb(a,b,d) values(2,2,2) // we can only combine DMLs with same column names. // all dmls should have same dmlOpType and same tableName. -func genDMLsWithSameCols(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) { +func genDMLsWithSameCols(op sqlmodel.DMLType, dmls []*sqlmodel.RowChange) ([]string, [][]interface{}) { queries := make([]string, 0, len(dmls)) args := make([][]interface{}, 0, len(dmls)) - var lastDML *DML - var query []string - var arg [][]interface{} - groupDMLs := make([]*DML, 0, len(dmls)) + var lastDML *sqlmodel.RowChange + var query string + var arg []interface{} + groupDMLs := make([]*sqlmodel.RowChange, 0, len(dmls)) // group dmls by same columns for i, dml := range dmls { if i == 0 { lastDML = dml } - if !sameColumns(lastDML, dml) { + if !sqlmodel.SameTypeTargetAndColumns(lastDML, dml) { query, arg = genSQLMultipleRows(op, groupDMLs) - queries = append(queries, query...) - args = append(args, arg...) + queries = append(queries, query) + args = append(args, arg) groupDMLs = groupDMLs[0:0] lastDML = dml @@ -1063,44 +374,53 @@ func genDMLsWithSameCols(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) } if len(groupDMLs) > 0 { query, arg = genSQLMultipleRows(op, groupDMLs) - queries = append(queries, query...) - args = append(args, arg...) + queries = append(queries, query) + args = append(args, arg) } return queries, args } // genDMLsWithSameTable groups and generates dmls with same table. // all the dmls should have same dmlOpType. -func genDMLsWithSameTable(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) { - queries := make([]string, 0, len(dmls)) - args := make([][]interface{}, 0, len(dmls)) +func genDMLsWithSameTable(op sqlmodel.DMLType, jobs []*job) ([]string, [][]interface{}) { + queries := make([]string, 0, len(jobs)) + args := make([][]interface{}, 0, len(jobs)) var lastTable string - groupDMLs := make([]*DML, 0, len(dmls)) + groupDMLs := make([]*sqlmodel.RowChange, 0, len(jobs)) // for updateDML, generate SQLs one by one - if op == updateDML { - for _, dml := range dmls { - query, arg := dml.genUpdateSQL() - queries = append(queries, query...) - args = append(args, arg...) + if op == sqlmodel.DMLUpdate { + for _, j := range jobs { + if j.safeMode { + query, arg := j.dml.GenSQL(sqlmodel.DMLDelete) + queries = append(queries, query) + args = append(args, arg) + query, arg = j.dml.GenSQL(sqlmodel.DMLReplace) + queries = append(queries, query) + args = append(args, arg) + continue + } + query, arg := j.dml.GenSQL(op) + queries = append(queries, query) + args = append(args, arg) } return queries, args } // group dmls with same table - for i, dml := range dmls { + for i, j := range jobs { if i == 0 { - lastTable = dml.targetTableID + lastTable = j.dml.TargetTableID() } - if lastTable != dml.targetTableID { + if lastTable != j.dml.TargetTableID() { query, arg := genDMLsWithSameCols(op, groupDMLs) queries = append(queries, query...) args = append(args, arg...) groupDMLs = groupDMLs[0:0] - lastTable = dml.targetTableID + lastTable = j.dml.TargetTableID() } - groupDMLs = append(groupDMLs, dml) + groupDMLs = append(groupDMLs, j.dml) } if len(groupDMLs) > 0 { query, arg := genDMLsWithSameCols(op, groupDMLs) @@ -1112,21 +432,34 @@ func genDMLsWithSameTable(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) // genDMLsWithSameOp groups and generates dmls by dmlOpType. // TODO: implement a volcano iterator interface for genDMLsWithSameXXX. -func genDMLsWithSameOp(dmls []*DML) ([]string, [][]interface{}) { - queries := make([]string, 0, len(dmls)) - args := make([][]interface{}, 0, len(dmls)) - var lastOp dmlOpType - groupDMLs := make([]*DML, 0, len(dmls)) +func genDMLsWithSameOp(jobs []*job) ([]string, [][]interface{}) { + queries := make([]string, 0, len(jobs)) + args := make([][]interface{}, 0, len(jobs)) + var lastOp sqlmodel.DMLType + jobsWithSameOp := make([]*job, 0, len(jobs)) // group dmls with same dmlOp - for i, dml := range dmls { - curOp := dmlOpType(dml.op) - if curOp == updateDML && !dml.updateIdentify() && !dml.safeMode { + for i, j := range jobs { + var curOp sqlmodel.DMLType + switch j.dml.Type() { + case sqlmodel.RowChangeUpdate: // if update statement didn't update identify values and not in safemode, regard it as insert on duplicate. - curOp = insertOnDuplicateDML - } else if curOp == insertDML && dml.safeMode { + if !j.dml.IsIdentityUpdated() && !j.safeMode { + curOp = sqlmodel.DMLInsertOnDuplicateUpdate + break + } + + curOp = sqlmodel.DMLUpdate + case sqlmodel.RowChangeInsert: // if insert with safemode, regard it as replace - curOp = replaceDML + if j.safeMode { + curOp = sqlmodel.DMLReplace + break + } + + curOp = sqlmodel.DMLInsert + case sqlmodel.RowChangeDelete: + curOp = sqlmodel.DMLDelete } if i == 0 { @@ -1135,17 +468,17 @@ func genDMLsWithSameOp(dmls []*DML) ([]string, [][]interface{}) { // now there are 5 situations: [insert, replace(insert with safemode), insert on duplicate(update without identify keys), update(update identify keys/update with safemode), delete] if lastOp != curOp { - query, arg := genDMLsWithSameTable(lastOp, groupDMLs) + query, arg := genDMLsWithSameTable(lastOp, jobsWithSameOp) queries = append(queries, query...) args = append(args, arg...) - groupDMLs = groupDMLs[0:0] + jobsWithSameOp = jobsWithSameOp[0:0] lastOp = curOp } - groupDMLs = append(groupDMLs, dml) + jobsWithSameOp = append(jobsWithSameOp, j) } - if len(groupDMLs) > 0 { - query, arg := genDMLsWithSameTable(lastOp, groupDMLs) + if len(jobsWithSameOp) > 0 { + query, arg := genDMLsWithSameTable(lastOp, jobsWithSameOp) queries = append(queries, query...) args = append(args, arg...) } diff --git a/dm/syncer/dml_test.go b/dm/syncer/dml_test.go index 148c2538eff..b6efe30695f 100644 --- a/dm/syncer/dml_test.go +++ b/dm/syncer/dml_test.go @@ -15,14 +15,15 @@ package syncer import ( "math" - "strings" + "testing" . "github.com/pingcap/check" + "github.com/stretchr/testify/require" - "github.com/pingcap/tiflow/dm/pkg/schema" - "github.com/pingcap/tiflow/dm/pkg/utils" + cdcmodel "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/pkg/sqlmodel" - "github.com/pingcap/tidb-tools/pkg/filter" tiddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" @@ -33,6 +34,14 @@ import ( "github.com/pingcap/tidb/util/mock" ) +var ( + location = binlog.Location{ + Position: binlog.MinPosition, + } + ec = &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} + ecWithSafeMode = &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location, safeMode: true} +) + func (s *testSyncerSuite) TestCastUnsigned(c *C) { // ref: https://dev.mysql.com/doc/refman/5.7/en/integer-types.html cases := []struct { @@ -70,420 +79,174 @@ func createTableInfo(p *parser.Parser, se sessionctx.Context, tableID int64, sql return tiddl.MockTableInfo(se, node.(*ast.CreateTableStmt), tableID) } -func (s *testSyncerSuite) TestFindFitIndex(c *C) { - p := parser.New() - se := mock.NewContext() - - ti, err := createTableInfo(p, se, 1, ` - create table t1( - a int, - b int, - c int, - d int not null, - primary key(a, b), - unique key(c), - unique key(d) - ); - `) - c.Assert(err, IsNil) - - columns := findFitIndex(ti) - c.Assert(columns, NotNil) - c.Assert(columns.Columns, HasLen, 2) - c.Assert(columns.Columns[0].Name.L, Equals, "a") - c.Assert(columns.Columns[1].Name.L, Equals, "b") - - ti, err = createTableInfo(p, se, 2, `create table t2(c int unique);`) - c.Assert(err, IsNil) - columns = findFitIndex(ti) - c.Assert(columns, IsNil) - - ti, err = createTableInfo(p, se, 3, `create table t3(d int not null unique);`) - c.Assert(err, IsNil) - columns = findFitIndex(ti) - c.Assert(columns, NotNil) - c.Assert(columns.Columns, HasLen, 1) - c.Assert(columns.Columns[0].Name.L, Equals, "d") - - ti, err = createTableInfo(p, se, 4, `create table t4(e int not null, key(e));`) - c.Assert(err, IsNil) - columns = findFitIndex(ti) - c.Assert(columns, IsNil) +func TestGenDMLWithSameOp(t *testing.T) { + targetTable1 := &cdcmodel.TableName{Schema: "db1", Table: "tb1"} + targetTable2 := &cdcmodel.TableName{Schema: "db2", Table: "tb2"} + sourceTable11 := &cdcmodel.TableName{Schema: "dba", Table: "tba"} + sourceTable12 := &cdcmodel.TableName{Schema: "dba", Table: "tbb"} + sourceTable21 := &cdcmodel.TableName{Schema: "dbb", Table: "tba"} + sourceTable22 := &cdcmodel.TableName{Schema: "dbb", Table: "tbb"} - ti, err = createTableInfo(p, se, 5, `create table t5(f datetime primary key);`) - c.Assert(err, IsNil) - columns = findFitIndex(ti) - c.Assert(columns, NotNil) - c.Assert(columns.Columns, HasLen, 1) - c.Assert(columns.Columns[0].Name.L, Equals, "f") + tableInfo11 := mockTableInfo(t, "create table db.tb(id int primary key, col1 int unique not null, name varchar(24))") + tableInfo12 := mockTableInfo(t, "create table db.tb(id int primary key, col1 int unique not null, name varchar(24))") + tableInfo21 := mockTableInfo(t, "create table db.tb(id int primary key, col2 int unique not null, name varchar(24))") + tableInfo22 := mockTableInfo(t, "create table db.tb(id int primary key, col3 int unique not null, name varchar(24))") - ti, err = createTableInfo(p, se, 6, `create table t6(g int primary key);`) - c.Assert(err, IsNil) - columns = findFitIndex(ti) - c.Assert(columns, NotNil) - c.Assert(columns.Columns, HasLen, 1) - c.Assert(columns.Columns[0].Name.L, Equals, "g") -} - -func (s *testSyncerSuite) TestGenMultipleKeys(c *C) { - p := parser.New() - se := mock.NewContext() - - testCases := []struct { - schema string - values []interface{} - keys []string - }{ - { - // test no keys - schema: `create table t1(a int)`, - values: []interface{}{10}, - keys: []string{"table"}, - }, - { - // one primary key - schema: `create table t2(a int primary key, b double)`, - values: []interface{}{60, 70.5}, - keys: []string{"60.a.table"}, - }, - { - // one unique key - schema: `create table t3(a int unique, b double)`, - values: []interface{}{60, 70.5}, - keys: []string{"60.a.table"}, - }, - { - // one ordinary key - schema: `create table t4(a int, b double, key(b))`, - values: []interface{}{60, 70.5}, - keys: []string{"table"}, - }, - { - // multiple keys - schema: `create table t5(a int, b text, c int, key(a), key(b(3)))`, - values: []interface{}{13, "abcdef", 15}, - keys: []string{"table"}, - }, - { - // multiple keys with primary key - schema: `create table t6(a int primary key, b varchar(16) unique)`, - values: []interface{}{16, "xyz"}, - keys: []string{"16.a.table", "xyz.b.table"}, - }, - { - // non-integer primary key - schema: `create table t65(a int unique, b varchar(16) primary key)`, - values: []interface{}{16, "xyz"}, - keys: []string{"xyz.b.table", "16.a.table"}, - }, - { - // primary key of multiple columns - schema: `create table t7(a int, b int, primary key(a, b))`, - values: []interface{}{59, 69}, - keys: []string{"59.a.69.b.table"}, - }, - { - // ordinary key of multiple columns - schema: `create table t75(a int, b int, c int, key(a, b), key(c, b))`, - values: []interface{}{48, 58, 68}, - keys: []string{"table"}, - }, - { - // so many keys - schema: ` - create table t8( - a int, b int, c int, - primary key(a, b), - unique key(b, c), - key(a, b, c), - unique key(c, a) - ) - `, - values: []interface{}{27, 37, 47}, - keys: []string{"27.a.37.b.table", "37.b.47.c.table", "47.c.27.a.table"}, - }, - { - // `null` for unique key - schema: ` - create table t8( - a int, b int default null, - primary key(a), - unique key(b) - ) - `, - values: []interface{}{17, nil}, - keys: []string{"17.a.table"}, - }, - } - sessCtx := utils.NewSessionCtx(map[string]string{"time_zone": "UTC"}) - for i, tc := range testCases { - schemaStr := tc.schema - assert := func(obtained interface{}, checker Checker, args ...interface{}) { - c.Assert(obtained, checker, append(args, Commentf("test case schema: %s", schemaStr))...) - } - - ti, err := createTableInfo(p, se, int64(i+1), tc.schema) - assert(err, IsNil) - dti := schema.GetDownStreamTI(ti, ti) - assert(dti, NotNil) - keys := genMultipleKeys(sessCtx, dti, ti, tc.values, "table") - assert(keys, DeepEquals, tc.keys) - } -} - -func (s *testSyncerSuite) TestGenWhere(c *C) { - p := parser.New() - se := mock.NewContext() - schema1 := "create table test.tb(id int primary key, col1 int unique not null, col2 int unique, name varchar(24))" - ti1, err := createTableInfo(p, se, 0, schema1) - c.Assert(err, IsNil) - schema2 := "create table test.tb(id int, col1 int, col2 int, name varchar(24))" - ti2, err := createTableInfo(p, se, 0, schema2) - c.Assert(err, IsNil) - ti1Index := &model.IndexInfo{ - Table: ti1.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti1.Columns[0].Name, - Offset: ti1.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - - testCases := []struct { - dml *DML - sql string - values []interface{} - }{ - { - newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti1.Columns, ti1, ti1Index, nil), - "`id` = ?", - []interface{}{1}, - }, - { - newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti1.Columns, ti1, ti1Index, nil), - "`id` = ?", - []interface{}{1}, - }, - { - newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti2.Columns, ti2, nil, nil), - "`id` = ? AND `col1` = ? AND `col2` = ? AND `name` = ?", - []interface{}{1, 2, 3, "haha"}, - }, - { - newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti2.Columns, ti2, nil, nil), - "`id` = ? AND `col1` = ? AND `col2` = ? AND `name` = ?", - []interface{}{1, 2, 3, "haha"}, - }, - } - - for _, tc := range testCases { - var buf strings.Builder - whereValues := tc.dml.genWhere(&buf) - c.Assert(buf.String(), Equals, tc.sql) - c.Assert(whereValues, DeepEquals, tc.values) - } -} - -func (s *testSyncerSuite) TestGenSQL(c *C) { - p := parser.New() - se := mock.NewContext() - schema := "create table test.tb(id int primary key, col1 int unique not null, col2 int unique, name varchar(24))" - ti, err := createTableInfo(p, se, 0, schema) - c.Assert(err, IsNil) - tiIndex := &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti.Columns[0].Name, - Offset: ti.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - - testCases := []struct { - dml *DML - queries []string - args [][]interface{} - }{ - { - newDML(insert, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, - [][]interface{}{{1, 2, 3, "haha"}}, - }, - { - newDML(insert, true, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"REPLACE INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, - [][]interface{}{{1, 2, 3, "haha"}}, - }, - { - newDML(del, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1"}, - [][]interface{}{{1}}, - }, - { - newDML(update, false, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"UPDATE `targetSchema`.`targetTable` SET `id` = ?, `col1` = ?, `col2` = ?, `name` = ? WHERE `id` = ? LIMIT 1"}, - [][]interface{}{{4, 5, 6, "hihi", 1}}, - }, - { - newDML(update, true, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1", "REPLACE INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, - [][]interface{}{{1}, {4, 5, 6, "hihi"}}, - }, - } - for _, tc := range testCases { - queries, args := tc.dml.genSQL() - c.Assert(queries, DeepEquals, tc.queries) - c.Assert(args, DeepEquals, tc.args) - } -} - -func (s *testSyncerSuite) TestValueHolder(c *C) { - holder := valuesHolder(0) - c.Assert(holder, Equals, "()") - holder = valuesHolder(10) - c.Assert(holder, Equals, "(?,?,?,?,?,?,?,?,?,?)") -} - -func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { - targetTableID1 := "`db1`.`tb1`" - targetTableID2 := "`db2`.`tb2`" - sourceTable11 := &filter.Table{Schema: "dba", Name: "tba"} - sourceTable12 := &filter.Table{Schema: "dba", Name: "tbb"} - sourceTable21 := &filter.Table{Schema: "dbb", Name: "tba"} - sourceTable22 := &filter.Table{Schema: "dbb", Name: "tbb"} - - p := parser.New() - se := mock.NewContext() - schema11 := "create table dba.tba(id int primary key, col1 int unique not null, name varchar(24))" - schema12 := "create table dba.tbb(id int primary key, col1 int unique not null, name varchar(24))" - schema21 := "create table dbb.tba(id int primary key, col2 int unique not null, name varchar(24))" - schema22 := "create table dbb.tbb(id int primary key, col3 int unique not null, name varchar(24))" - ti11, err := createTableInfo(p, se, 0, schema11) - c.Assert(err, IsNil) - ti11Index := &model.IndexInfo{ - Table: ti11.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti11.Columns[0].Name, - Offset: ti11.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi11 := &schema.DownstreamTableInfo{ - AbsoluteUKIndexInfo: ti11Index, - } - ti12, err := createTableInfo(p, se, 0, schema12) - c.Assert(err, IsNil) - ti12Index := &model.IndexInfo{ - Table: ti12.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti12.Columns[0].Name, - Offset: ti12.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi12 := &schema.DownstreamTableInfo{ - AbsoluteUKIndexInfo: ti12Index, - } - ti21, err := createTableInfo(p, se, 0, schema21) - c.Assert(err, IsNil) - ti21Index := &model.IndexInfo{ - Table: ti21.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti21.Columns[0].Name, - Offset: ti21.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi21 := &schema.DownstreamTableInfo{ - AbsoluteUKIndexInfo: ti21Index, - } - ti22, err := createTableInfo(p, se, 0, schema22) - c.Assert(err, IsNil) - ti22Index := &model.IndexInfo{ - Table: ti22.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti22.Columns[0].Name, - Offset: ti22.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } - downTi22 := &schema.DownstreamTableInfo{ - AbsoluteUKIndexInfo: ti22Index, - } - - dmls := []*DML{ + dmls := []*job{ // insert - newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(insert, true, targetTableID1, sourceTable12, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, nil, []interface{}{1, 1, "a"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, nil, []interface{}{2, 2, "b"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, nil, []interface{}{3, 3, "c"}, tableInfo12, nil, nil), + ecWithSafeMode, + ), + // update no index but safemode - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, tableInfo12, nil, nil), + ecWithSafeMode, + ), + // update uk - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, tableInfo12, nil, nil), + ecWithSafeMode, + ), + // update pk - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, tableInfo12, nil, nil), + ecWithSafeMode, + ), // delete - newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(del, true, targetTableID1, sourceTable12, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{4, 4, "aa"}, nil, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{5, 5, "bb"}, nil, tableInfo11, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, []interface{}{6, 6, "cc"}, nil, tableInfo12, nil, nil), + ecWithSafeMode, + ), // target table 2 // insert - newDML(insert, true, targetTableID2, sourceTable21, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(insert, false, targetTableID2, sourceTable21, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(insert, false, targetTableID2, sourceTable22, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti22.Columns, ti22, ti22Index, downTi22), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, nil, []interface{}{1, 1, "a"}, tableInfo21, nil, nil), + ecWithSafeMode, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, nil, []interface{}{2, 2, "b"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable22, targetTable2, nil, []interface{}{3, 3, "c"}, tableInfo22, nil, nil), + ec, + ), + // update no index - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable22, targetTable2, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, tableInfo22, nil, nil), + ec, + ), + // update uk - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable22, targetTable2, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, tableInfo22, nil, nil), + ec, + ), + // update pk - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable22, targetTable2, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, tableInfo22, nil, nil), + ec, + ), + // delete - newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), - newDML(del, false, targetTableID2, sourceTable22, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{4, 4, "aa"}, nil, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable21, targetTable2, []interface{}{5, 5, "bb"}, nil, tableInfo21, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable22, targetTable2, []interface{}{6, 6, "cc"}, nil, tableInfo22, nil, nil), + ec, + ), // table1 // detele - newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{44, 44, "aaa"}, nil, []interface{}{44, 44, "aaa"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{55, 55, "bbb"}, nil, []interface{}{55, 55, "bbb"}, ti11.Columns, ti11, ti11Index, downTi11), - newDML(del, false, targetTableID1, sourceTable12, nil, []interface{}{66, 66, "ccc"}, nil, []interface{}{66, 66, "ccc"}, ti12.Columns, ti12, ti12Index, downTi12), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{44, 44, "aaa"}, nil, tableInfo11, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable11, targetTable1, []interface{}{55, 55, "bbb"}, nil, tableInfo11, nil, nil), + ec, + ), + newDMLJob( + sqlmodel.NewRowChange(sourceTable12, targetTable1, []interface{}{66, 66, "ccc"}, nil, tableInfo12, nil, nil), + ec, + ), } expectQueries := []string{ @@ -567,70 +330,8 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { } queries, args := genDMLsWithSameOp(dmls) - c.Assert(queries, DeepEquals, expectQueries) - c.Assert(args, DeepEquals, expectArgs) -} - -func (s *testSyncerSuite) TestTruncateIndexValues(c *C) { - p := parser.New() - se := mock.NewContext() - - testCases := []struct { - schema string - values []interface{} - preValues []interface{} - }{ - { - // test not prefix key - schema: `create table t1(a int, b varchar(20), unique key b(b))`, - values: []interface{}{10, "1234"}, - preValues: []interface{}{"1234"}, - }, - { - // test not string key - schema: `create table t1(a int, b text, unique key a(a))`, - values: []interface{}{10, "1234"}, - preValues: []interface{}{int64(10)}, - }, - { - // test keys - schema: `create table t1(a int, b text, unique key b(b(3)))`, - values: []interface{}{10, "1234"}, - preValues: []interface{}{"123"}, - }, - { - // test multi keys - schema: `create table t1(a int, b text, unique key c2(a, b(3)))`, - values: []interface{}{10, "1234"}, - preValues: []interface{}{int64(10), "123"}, - }, - { - // value is nil - schema: `create table t1(a int, b text, unique key c2(a, b(3)))`, - values: []interface{}{10, nil}, - preValues: []interface{}{int64(10), nil}, - }, - } - sessCtx := utils.NewSessionCtx(map[string]string{"time_zone": "UTC"}) - for i, tc := range testCases { - schemaStr := tc.schema - assert := func(obtained interface{}, checker Checker, args ...interface{}) { - c.Assert(obtained, checker, append(args, Commentf("test case schema: %s", schemaStr))...) - } - ti, err := createTableInfo(p, se, int64(i+1), tc.schema) - assert(err, IsNil) - dti := schema.GetDownStreamTI(ti, ti) - assert(dti, NotNil) - assert(dti.AvailableUKIndexList, NotNil) - cols := make([]*model.ColumnInfo, 0, len(dti.AvailableUKIndexList[0].Columns)) - values := make([]interface{}, 0, len(dti.AvailableUKIndexList[0].Columns)) - for _, column := range dti.AvailableUKIndexList[0].Columns { - cols = append(cols, ti.Columns[column.Offset]) - values = append(values, tc.values[column.Offset]) - } - realPreValue := truncateIndexValues(sessCtx, ti, dti.AvailableUKIndexList[0], cols, values) - assert(realPreValue, DeepEquals, tc.preValues) - } + require.Equal(t, expectQueries, queries) + require.Equal(t, expectArgs, args) } func (s *testSyncerSuite) TestGBKExtractValueFromData(c *C) { diff --git a/dm/syncer/dml_worker.go b/dm/syncer/dml_worker.go index 78c4e48dd1f..016a418969a 100644 --- a/dm/syncer/dml_worker.go +++ b/dm/syncer/dml_worker.go @@ -18,7 +18,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb-tools/pkg/filter" "go.uber.org/zap" tcontext "github.com/pingcap/tiflow/dm/pkg/context" @@ -27,6 +26,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/syncer/dbconn" "github.com/pingcap/tiflow/dm/syncer/metrics" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) // DMLWorker is used to sync dml. @@ -46,10 +46,10 @@ type DMLWorker struct { // callback func // TODO: refine callback func - successFunc func(int, int, []*job) - fatalFunc func(*job, error) - lagFunc func(*job, int) - addCountFunc func(bool, string, opType, int64, *filter.Table) + successFunc func(int, int, []*job) + fatalFunc func(*job, error) + lagFunc func(*job, int) + updateJobMetricsFunc func(bool, string, *job) // channel inCh chan *job @@ -63,22 +63,22 @@ func dmlWorkerWrap(inCh chan *job, syncer *Syncer) chan *job { chanSize /= 2 } dmlWorker := &DMLWorker{ - batch: syncer.cfg.Batch, - workerCount: syncer.cfg.WorkerCount, - chanSize: chanSize, - multipleRows: syncer.cfg.MultipleRows, - task: syncer.cfg.Name, - source: syncer.cfg.SourceID, - worker: syncer.cfg.WorkerName, - logger: syncer.tctx.Logger.WithFields(zap.String("component", "dml_worker")), - successFunc: syncer.successFunc, - fatalFunc: syncer.fatalFunc, - lagFunc: syncer.updateReplicationJobTS, - addCountFunc: syncer.addCount, - tctx: syncer.tctx, - toDBConns: syncer.toDBConns, - inCh: inCh, - flushCh: make(chan *job), + batch: syncer.cfg.Batch, + workerCount: syncer.cfg.WorkerCount, + chanSize: chanSize, + multipleRows: syncer.cfg.MultipleRows, + task: syncer.cfg.Name, + source: syncer.cfg.SourceID, + worker: syncer.cfg.WorkerName, + logger: syncer.tctx.Logger.WithFields(zap.String("component", "dml_worker")), + successFunc: syncer.successFunc, + fatalFunc: syncer.fatalFunc, + lagFunc: syncer.updateReplicationJobTS, + updateJobMetricsFunc: syncer.updateJobMetrics, + tctx: syncer.tctx, + toDBConns: syncer.toDBConns, + inCh: inCh, + flushCh: make(chan *job), } go func() { @@ -117,25 +117,25 @@ func (w *DMLWorker) run() { metrics.QueueSizeGauge.WithLabelValues(w.task, "dml_worker_input", w.source).Set(float64(len(w.inCh))) switch j.tp { case flush: - w.addCountFunc(false, adminQueueName, j.tp, 1, j.targetTable) + w.updateJobMetricsFunc(false, adminQueueName, j) w.sendJobToAllDmlQueue(j, jobChs, queueBucketMapping) j.flushWg.Wait() - w.addCountFunc(true, adminQueueName, j.tp, 1, j.targetTable) + w.updateJobMetricsFunc(true, adminQueueName, j) w.flushCh <- j case asyncFlush: - w.addCountFunc(false, adminQueueName, j.tp, 1, j.targetTable) + w.updateJobMetricsFunc(false, adminQueueName, j) w.sendJobToAllDmlQueue(j, jobChs, queueBucketMapping) w.flushCh <- j case conflict: - w.addCountFunc(false, adminQueueName, j.tp, 1, j.targetTable) + w.updateJobMetricsFunc(false, adminQueueName, j) w.sendJobToAllDmlQueue(j, jobChs, queueBucketMapping) j.flushWg.Wait() - w.addCountFunc(true, adminQueueName, j.tp, 1, j.targetTable) + w.updateJobMetricsFunc(true, adminQueueName, j) default: - queueBucket := int(utils.GenHashKey(j.dml.key)) % w.workerCount - w.addCountFunc(false, queueBucketMapping[queueBucket], j.tp, 1, j.targetTable) + queueBucket := int(utils.GenHashKey(j.dmlQueueKey)) % w.workerCount + w.updateJobMetricsFunc(false, queueBucketMapping[queueBucket], j) startTime := time.Now() - w.logger.Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", j.dml.key)) + w.logger.Debug("queue for key", zap.Int("queue", queueBucket), zap.String("key", j.dmlQueueKey)) jobChs[queueBucket] <- j metrics.AddJobDurationHistogram.WithLabelValues(j.tp.String(), w.task, queueBucketMapping[queueBucket], w.source).Observe(time.Since(startTime).Seconds()) } @@ -201,7 +201,7 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { args [][]interface{} db = w.toDBConns[queueID] err error - dmls = make([]*DML, 0, len(jobs)) + dmls = make([]*sqlmodel.RowChange, 0, len(jobs)) ) defer func() { @@ -238,10 +238,7 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { } }) - for _, j := range jobs { - dmls = append(dmls, j.dml) - } - queries, args = w.genSQLs(dmls) + queries, args = w.genSQLs(jobs) failpoint.Inject("WaitUserCancel", func(v failpoint.Value) { t := v.(int) time.Sleep(time.Duration(t) * time.Second) @@ -266,17 +263,43 @@ func (w *DMLWorker) executeBatchJobs(queueID int, jobs []*job) { } // genSQLs generate SQLs in single row mode or multiple rows mode. -func (w *DMLWorker) genSQLs(dmls []*DML) ([]string, [][]interface{}) { +func (w *DMLWorker) genSQLs(jobs []*job) ([]string, [][]interface{}) { if w.multipleRows { - return genDMLsWithSameOp(dmls) + return genDMLsWithSameOp(jobs) } - queries := make([]string, 0, len(dmls)) - args := make([][]interface{}, 0, len(dmls)) - for _, dml := range dmls { - query, arg := dml.genSQL() - queries = append(queries, query...) - args = append(args, arg...) + queries := make([]string, 0, len(jobs)) + args := make([][]interface{}, 0, len(jobs)) + for _, j := range jobs { + var query string + var arg []interface{} + appendQueryAndArg := func() { + queries = append(queries, query) + args = append(args, arg) + } + + switch j.dml.Type() { + case sqlmodel.RowChangeInsert: + if j.safeMode { + query, arg = j.dml.GenSQL(sqlmodel.DMLReplace) + } else { + query, arg = j.dml.GenSQL(sqlmodel.DMLInsert) + } + + case sqlmodel.RowChangeUpdate: + if j.safeMode { + query, arg = j.dml.GenSQL(sqlmodel.DMLDelete) + appendQueryAndArg() + query, arg = j.dml.GenSQL(sqlmodel.DMLReplace) + } else { + query, arg = j.dml.GenSQL(sqlmodel.DMLUpdate) + } + + case sqlmodel.RowChangeDelete: + query, arg = j.dml.GenSQL(sqlmodel.DMLDelete) + } + + appendQueryAndArg() } return queries, args } diff --git a/dm/syncer/dml_worker_test.go b/dm/syncer/dml_worker_test.go new file mode 100644 index 00000000000..ffb51a553ab --- /dev/null +++ b/dm/syncer/dml_worker_test.go @@ -0,0 +1,113 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package syncer + +import ( + "testing" + + tiddl "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + timodel "github.com/pingcap/tidb/parser/model" + timock "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" + + cdcmodel "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/sqlmodel" +) + +func mockTableInfo(t *testing.T, sql string) *timodel.TableInfo { + t.Helper() + + p := parser.New() + se := timock.NewContext() + node, err := p.ParseOneStmt(sql, "", "") + require.NoError(t, err) + ti, err := tiddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 1) + require.NoError(t, err) + return ti +} + +func TestGenSQL(t *testing.T) { + t.Parallel() + + source := &cdcmodel.TableName{Schema: "db", Table: "tb"} + target := &cdcmodel.TableName{Schema: "targetSchema", Table: "targetTable"} + createSQL := "create table db.tb(id int primary key, col1 int unique not null, col2 int unique, name varchar(24))" + + cases := []struct { + preValues []interface{} + postValues []interface{} + safeMode bool + + expectedSQLs []string + expectedArgs [][]interface{} + }{ + { + nil, + []interface{}{1, 2, 3, "haha"}, + false, + + []string{"INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, + [][]interface{}{{1, 2, 3, "haha"}}, + }, + { + nil, + []interface{}{1, 2, 3, "haha"}, + true, + + []string{"REPLACE INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, + [][]interface{}{{1, 2, 3, "haha"}}, + }, + { + []interface{}{1, 2, 3, "haha"}, + nil, + false, + + []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1"}, + [][]interface{}{{1}}, + }, + { + []interface{}{1, 2, 3, "haha"}, + []interface{}{4, 5, 6, "hihi"}, + false, + + []string{"UPDATE `targetSchema`.`targetTable` SET `id` = ?, `col1` = ?, `col2` = ?, `name` = ? WHERE `id` = ? LIMIT 1"}, + [][]interface{}{{4, 5, 6, "hihi", 1}}, + }, + { + []interface{}{1, 2, 3, "haha"}, + []interface{}{4, 5, 6, "hihi"}, + true, + + []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1", "REPLACE INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, + [][]interface{}{{1}, {4, 5, 6, "hihi"}}, + }, + } + + worker := &DMLWorker{} + + for _, c := range cases { + tableInfo := mockTableInfo(t, createSQL) + change := sqlmodel.NewRowChange(source, target, c.preValues, c.postValues, tableInfo, nil, nil) + testEC := ec + if c.safeMode { + testEC = ecWithSafeMode + } + dmlJob := newDMLJob(change, testEC) + queries, args := worker.genSQLs([]*job{dmlJob}) + require.Equal(t, c.expectedSQLs, queries) + require.Equal(t, c.expectedArgs, args) + } +} diff --git a/dm/syncer/job.go b/dm/syncer/job.go index d6161b587da..28f6efbbb43 100644 --- a/dm/syncer/job.go +++ b/dm/syncer/job.go @@ -22,15 +22,14 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) type opType byte const ( null opType = iota - insert - update - del + dml ddl xid flush @@ -44,12 +43,8 @@ const ( func (t opType) String() string { switch t { - case insert: - return "insert" - case update: - return "update" - case del: - return "delete" + case dml: + return "dml" case ddl: return "ddl" case xid: @@ -83,7 +78,9 @@ type job struct { // sql example: drop table `s1`.`t1`, `s2`.`t2`. sourceTbls map[string][]*filter.Table targetTable *filter.Table - dml *DML + dml *sqlmodel.RowChange + dmlQueueKey string + safeMode bool retry bool location binlog.Location // location of last received (ROTATE / QUERY / XID) event, for global/table checkpoint startLocation binlog.Location // start location of the sql in binlog, for handle_error @@ -109,16 +106,24 @@ func (j *job) String() string { if j.dml != nil { dmlStr = j.dml.String() } - return fmt.Sprintf("tp: %s, flushSeq: %d, dml: %s, ddls: %s, last_location: %s, start_location: %s, current_location: %s", j.tp, j.flushSeq, dmlStr, j.ddls, j.location, j.startLocation, j.currentLocation) + return fmt.Sprintf("tp: %s, flushSeq: %d, dml: [%s], safemode: %v, ddls: %s, last_location: %s, start_location: %s, current_location: %s", j.tp, j.flushSeq, dmlStr, j.safeMode, j.ddls, j.location, j.startLocation, j.currentLocation) } -func newDMLJob(tp opType, sourceTable, targetTable *filter.Table, dml *DML, ec *eventContext) *job { +func newDMLJob(rowChange *sqlmodel.RowChange, ec *eventContext) *job { + sourceTable := rowChange.GetSourceTable() + targetTable := rowChange.GetTargetTable() + // TODO: remove sourceTbls and targetTable for dml Job return &job{ - tp: tp, - sourceTbls: map[string][]*filter.Table{sourceTable.Schema: {sourceTable}}, - targetTable: targetTable, - dml: dml, + tp: dml, + sourceTbls: map[string][]*filter.Table{ + sourceTable.Schema: { + &filter.Table{Schema: sourceTable.Schema, Name: sourceTable.Table}, + }, + }, + targetTable: &filter.Table{Schema: targetTable.Schema, Name: targetTable.Table}, + dml: rowChange, retry: true, + safeMode: ec.safeMode, location: *ec.lastLocation, startLocation: *ec.startLocation, @@ -228,6 +233,14 @@ func newConflictJob(workerCount int) *job { } } +// newCompactJob is only used for metrics. +func newCompactJob(targetTable *filter.Table) *job { + return &job{ + tp: compact, + targetTable: targetTable, + } +} + // put queues into bucket to monitor them. func queueBucketName(queueID int) string { return fmt.Sprintf("q_%d", queueID%defaultBucketCount) diff --git a/dm/syncer/job_test.go b/dm/syncer/job_test.go index 8b839e189cc..46abaed1955 100644 --- a/dm/syncer/job_test.go +++ b/dm/syncer/job_test.go @@ -14,14 +14,15 @@ package syncer import ( + "testing" + . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" - "github.com/pingcap/tidb/parser" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/types" - "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" + cdcmodel "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) var _ = Suite(&testJobSuite{}) @@ -37,14 +38,8 @@ func (t *testJobSuite) TestJobTypeString(c *C) { null, "", }, { - insert, - "insert", - }, { - update, - "update", - }, { - del, - "delete", + dml, + "dml", }, { ddl, "ddl", @@ -69,14 +64,16 @@ func (t *testJobSuite) TestJobTypeString(c *C) { } } -func (t *testJobSuite) TestJob(c *C) { +func TestJob(t *testing.T) { + t.Parallel() + ddlInfo := &ddlInfo{ sourceTables: []*filter.Table{{Schema: "test1", Name: "t1"}}, targetTables: []*filter.Table{{Schema: "test2", Name: "t2"}}, } - table := &filter.Table{Schema: "test", Name: "t1"} + table := &cdcmodel.TableName{Schema: "test", Table: "t1"} location := binlog.NewLocation("") - ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} + ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location, safeMode: true} qec := &queryEventContext{ eventContext: ec, originSQL: "create database test", @@ -85,47 +82,34 @@ func (t *testJobSuite) TestJob(c *C) { } schema := "create table test.tb(id int primary key, col1 int unique not null)" - p := parser.New() - se := mock.NewContext() - ti, err := createTableInfo(p, se, 0, schema) - c.Assert(err, IsNil) - tiIndex := &model.IndexInfo{ - Table: ti.Name, - Unique: true, - Primary: true, - State: model.StatePublic, - Tp: model.IndexTypeBtree, - Columns: []*model.IndexColumn{{ - Name: ti.Columns[0].Name, - Offset: ti.Columns[0].Offset, - Length: types.UnspecifiedLength, - }}, - } + ti := mockTableInfo(t, schema) + + exampleChange := sqlmodel.NewRowChange(table, nil, nil, []interface{}{2, 2}, ti, nil, nil) testCases := []struct { job *job jobStr string }{ { - newDMLJob(insert, table, table, newDML(insert, true, "targetTable", table, nil, []interface{}{2, 2}, nil, []interface{}{2, 2}, ti.Columns, ti, tiIndex, nil), ec), - "tp: insert, flushSeq: 0, dml: [safemode: true, targetTableID: targetTable, op: insert, columns: [id col1], oldValues: [], values: [2 2]], ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", + newDMLJob(exampleChange, ec), + "tp: dml, flushSeq: 0, dml: [type: ChangeInsert, source table: test.t1, target table: test.t1, preValues: [], postValues: [2 2]], safemode: true, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newDDLJob(qec), - "tp: ddl, flushSeq: 0, dml: , ddls: [create database test], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", + "tp: ddl, flushSeq: 0, dml: [], safemode: false, ddls: [create database test], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newXIDJob(binlog.NewLocation(""), binlog.NewLocation(""), binlog.NewLocation("")), - "tp: xid, flushSeq: 0, dml: , ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", + "tp: xid, flushSeq: 0, dml: [], safemode: false, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newFlushJob(16, 1), - "tp: flush, flushSeq: 1, dml: , ddls: [], last_location: position: (, 0), gtid-set: , start_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", + "tp: flush, flushSeq: 1, dml: [], safemode: false, ddls: [], last_location: position: (, 0), gtid-set: , start_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", }, { newSkipJob(ec), - "tp: skip, flushSeq: 0, dml: , ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", + "tp: skip, flushSeq: 0, dml: [], safemode: false, ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 0), gtid-set: , current_location: position: (, 0), gtid-set: ", }, } for _, testCase := range testCases { - c.Assert(testCase.job.String(), Equals, testCase.jobStr) + require.Equal(t, testCase.jobStr, testCase.job.String()) } } diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 4cd21535bde..9f005e7e993 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -73,6 +73,7 @@ import ( sm "github.com/pingcap/tiflow/dm/syncer/safe-mode" "github.com/pingcap/tiflow/dm/syncer/shardddl" "github.com/pingcap/tiflow/pkg/errorutil" + "github.com/pingcap/tiflow/pkg/sqlmodel" ) var ( @@ -432,11 +433,11 @@ func (s *Syncer) Init(ctx context.Context) (err error) { } } s.checkpointFlushWorker = &checkpointFlushWorker{ - input: nil, // will be created in s.reset() - cp: s.checkpoint, - execError: &s.execError, - afterFlushFn: s.afterFlushCheckpoint, - addCountFunc: s.addCount, + input: nil, // will be created in s.reset() + cp: s.checkpoint, + execError: &s.execError, + afterFlushFn: s.afterFlushCheckpoint, + updateJobMetricsFn: s.updateJobMetrics, } // when Init syncer, set active relay log info @@ -807,19 +808,34 @@ func (s *Syncer) trackTableInfoFromDownstream(tctx *tcontext.Context, sourceTabl return nil } -func (s *Syncer) addCount(isFinished bool, queueBucket string, tp opType, n int64, targetTable *filter.Table) { +var dmlMetric = map[sqlmodel.RowChangeType]string{ + sqlmodel.RowChangeInsert: "insert", + sqlmodel.RowChangeUpdate: "update", + sqlmodel.RowChangeDelete: "delete", +} + +func (s *Syncer) updateJobMetrics(isFinished bool, queueBucket string, j *job) { + tp := j.tp + targetTable := j.targetTable + count := 1 + if tp == ddl { + count = len(j.ddls) + } + m := metrics.AddedJobsTotal if isFinished { - s.count.Add(n) + s.count.Add(int64(count)) m = metrics.FinishedJobsTotal } switch tp { - case insert, update, del, ddl, flush, asyncFlush, conflict, compact: - m.WithLabelValues(tp.String(), s.cfg.Name, queueBucket, s.cfg.SourceID, s.cfg.WorkerName, targetTable.Schema, targetTable.Name).Add(float64(n)) + case dml: + m.WithLabelValues(dmlMetric[j.dml.Type()], s.cfg.Name, queueBucket, s.cfg.SourceID, s.cfg.WorkerName, targetTable.Schema, targetTable.Name).Add(float64(count)) + case ddl, flush, asyncFlush, conflict, compact: + m.WithLabelValues(tp.String(), s.cfg.Name, queueBucket, s.cfg.SourceID, s.cfg.WorkerName, targetTable.Schema, targetTable.Name).Add(float64(count)) case skip, xid: // ignore skip/xid jobs default: - s.tctx.L().Warn("unknown job operation type", zap.Stringer("type", tp)) + s.tctx.L().Warn("unknown job operation type", zap.Stringer("type", j.tp)) } } @@ -901,7 +917,7 @@ var ( // There should not be a second way to send jobs to DML queue or DDL queue. func (s *Syncer) addJob(job *job) { failpoint.Inject("countJobFromOneEvent", func() { - if job.tp == insert { + if job.tp == dml { if job.currentLocation.Position.Compare(lastLocation.Position) == 0 { lastLocationNum++ } else { @@ -921,7 +937,7 @@ func (s *Syncer) addJob(job *job) { } }) failpoint.Inject("countJobFromOneGTID", func() { - if job.tp == insert { + if job.tp == dml { if binlog.CompareLocation(job.currentLocation, lastLocation, true) == 0 { lastLocationNum++ } else { @@ -952,12 +968,12 @@ func (s *Syncer) addJob(job *job) { s.jobWg.Add(1) s.dmlJobCh <- job case ddl: - s.addCount(false, adminQueueName, job.tp, 1, job.targetTable) + s.updateJobMetrics(false, adminQueueName, job) s.jobWg.Add(1) startTime := time.Now() s.ddlJobCh <- job metrics.AddJobDurationHistogram.WithLabelValues("ddl", s.cfg.Name, adminQueueName, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) - case insert, update, del: + case dml: s.dmlJobCh <- job failpoint.Inject("checkCheckpointInMiddleOfTransaction", func() { s.tctx.L().Info("receive dml job", zap.Any("dml job", job)) @@ -1033,7 +1049,7 @@ func (s *Syncer) handleJob(job *job) (added2Queue bool, err error) { // 3. after job is sent to queue switch job.tp { - case insert, update, del: + case dml: // DMl events will generate many jobs and only caller knows all jobs has been sent, so many logics are done by // caller s.isTransactionEnd = false @@ -1370,7 +1386,7 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. continue } s.jobWg.Done() - s.addCount(true, queueBucket, ddlJob.tp, int64(len(ddlJob.ddls)), ddlJob.targetTable) + s.updateJobMetrics(true, queueBucket, ddlJob) } } @@ -1384,7 +1400,7 @@ func (s *Syncer) successFunc(queueID int, statementsCnt int, jobs []*job) { switch j.tp { case ddl: metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageDDLExec, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(j.jobAddTime).Seconds()) - case insert, update, del: + case dml: metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageDMLExec, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(j.jobAddTime).Seconds()) // metric only increases by 1 because dm batches sql jobs in a single transaction. metrics.FinishedTransactionTotal.WithLabelValues(s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Inc() @@ -1392,7 +1408,7 @@ func (s *Syncer) successFunc(queueID int, statementsCnt int, jobs []*job) { } for _, sqlJob := range jobs { - s.addCount(true, queueBucket, sqlJob.tp, 1, sqlJob.targetTable) + s.updateJobMetrics(true, queueBucket, sqlJob) } s.updateReplicationJobTS(nil, dmlWorkerJobIdx(queueID)) metrics.ReplicationTransactionBatch.WithLabelValues(s.cfg.WorkerName, s.cfg.Name, s.cfg.SourceID, queueBucket, "statements").Observe(float64(statementsCnt)) @@ -2256,26 +2272,12 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err } extRows := generateExtendColumn(originRows, s.tableRouter, sourceTable, s.cfg.SourceID) - rows := originRows - if extRows != nil { - rows = extRows - } - - prunedColumns, prunedRows, err := pruneGeneratedColumnDML(tableInfo, rows) - if err != nil { - return err - } - var ( - dmls []*DML - jobType opType - ) + var dmls []*sqlmodel.RowChange param := &genDMLParam{ - targetTableID: utils.GenTableID(targetTable), - data: prunedRows, + targetTable: targetTable, originalData: originRows, - columns: prunedColumns, sourceTableInfo: tableInfo, sourceTable: sourceTable, extendData: extRows, @@ -2294,7 +2296,6 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err return terror.Annotatef(err, "gen insert sqls failed, sourceTable: %v, targetTable: %v", sourceTable, targetTable) } metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageGenWriteRows, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(ec.startTime).Seconds()) - jobType = insert case replication.UPDATE_ROWS_EVENTv0, replication.UPDATE_ROWS_EVENTv1, replication.UPDATE_ROWS_EVENTv2: oldExprFilter, newExprFilter, err2 := s.exprFilterGroup.GetUpdateExprs(sourceTable, tableInfo) @@ -2308,7 +2309,6 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err return terror.Annotatef(err, "gen update sqls failed, sourceTable: %v, targetTable: %v", sourceTable, targetTable) } metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageGenUpdateRows, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(ec.startTime).Seconds()) - jobType = update case replication.DELETE_ROWS_EVENTv0, replication.DELETE_ROWS_EVENTv1, replication.DELETE_ROWS_EVENTv2: exprFilter, err2 := s.exprFilterGroup.GetDeleteExprs(sourceTable, tableInfo) @@ -2321,7 +2321,6 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err return terror.Annotatef(err, "gen delete sqls failed, sourceTable: %v, targetTable: %v", sourceTable, targetTable) } metrics.BinlogEventCost.WithLabelValues(metrics.BinlogEventCostStageGenDeleteRows, s.cfg.Name, s.cfg.WorkerName, s.cfg.SourceID).Observe(time.Since(ec.startTime).Seconds()) - jobType = del default: ec.tctx.L().Debug("ignoring unrecognized event", zap.String("event", "row"), zap.Stringer("type", ec.header.EventType)) @@ -2329,14 +2328,17 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err } startTime := time.Now() + + metricTp := "" for i := range dmls { - job := newDMLJob(jobType, sourceTable, targetTable, dmls[i], &ec) + metricTp = dmlMetric[dmls[i].Type()] + job := newDMLJob(dmls[i], &ec) added2Queue, err2 := s.handleJobFunc(job) if err2 != nil || !added2Queue { return err2 } } - metrics.DispatchBinlogDurationHistogram.WithLabelValues(jobType.String(), s.cfg.Name, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) + metrics.DispatchBinlogDurationHistogram.WithLabelValues(metricTp, s.cfg.Name, s.cfg.SourceID).Observe(time.Since(startTime).Seconds()) if len(sourceTable.Schema) != 0 { // when in position-based replication, now events before table checkpoint is sent to queue. But in GTID-based diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index 1eac0b668fa..c555fb766c6 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/syncer/dbconn" "github.com/pingcap/tiflow/pkg/errorutil" + "github.com/pingcap/tiflow/pkg/sqlmodel" sqlmock "github.com/DATA-DOG/go-sqlmock" "github.com/go-mysql-org/go-mysql/mysql" @@ -841,11 +842,11 @@ func (s *testSyncerSuite) TestRun(c *C) { streamer: mockStreamer, } syncer.checkpointFlushWorker = &checkpointFlushWorker{ - input: make(chan *checkpointFlushTask, 16), - cp: syncer.checkpoint, - execError: &syncer.execError, - afterFlushFn: syncer.afterFlushCheckpoint, - addCountFunc: func(bool, string, opType, int64, *filter.Table) {}, + input: make(chan *checkpointFlushTask, 16), + cp: syncer.checkpoint, + execError: &syncer.execError, + afterFlushFn: syncer.afterFlushCheckpoint, + updateJobMetricsFn: func(bool, string, *job) {}, } syncer.handleJobFunc = syncer.addJobToMemory @@ -882,7 +883,7 @@ func (s *testSyncerSuite) TestRun(c *C) { []string{"CREATE TABLE IF NOT EXISTS `test_1`.`t_2` (`id` INT PRIMARY KEY,`name` VARCHAR(24))"}, nil, }, { - insert, + dml, []string{"REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(580981944116838401), "a"}}, }, { @@ -894,16 +895,16 @@ func (s *testSyncerSuite) TestRun(c *C) { []string{"ALTER TABLE `test_1`.`t_1` ADD INDEX `index1`(`name`)"}, nil, }, { - insert, + dml, []string{"REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(580981944116838402), "b"}}, }, { - del, + dml, []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1"}, [][]interface{}{{int64(580981944116838401)}}, }, { // safe mode is true, will split update to delete + replace - update, + dml, []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1", "REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(580981944116838402)}, {int64(580981944116838401), "b"}}, }, { @@ -984,11 +985,11 @@ func (s *testSyncerSuite) TestRun(c *C) { streamer: mockStreamer, } syncer.checkpointFlushWorker = &checkpointFlushWorker{ - input: make(chan *checkpointFlushTask, 16), - cp: syncer.checkpoint, - execError: &syncer.execError, - afterFlushFn: syncer.afterFlushCheckpoint, - addCountFunc: func(bool, string, opType, int64, *filter.Table) {}, + input: make(chan *checkpointFlushTask, 16), + cp: syncer.checkpoint, + execError: &syncer.execError, + afterFlushFn: syncer.afterFlushCheckpoint, + updateJobMetricsFn: func(bool, string, *job) {}, } // When crossing safeModeExitPoint, will generate a flush sql @@ -1000,11 +1001,11 @@ func (s *testSyncerSuite) TestRun(c *C) { expectJobs2 := []*expectJob{ { - insert, + dml, []string{"INSERT INTO `test_1`.`t_2` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(3), "c"}}, }, { - del, + dml, []string{"DELETE FROM `test_1`.`t_2` WHERE `id` = ? LIMIT 1"}, [][]interface{}{{int64(3)}}, }, { @@ -1122,11 +1123,11 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { streamer: mockStreamer, } syncer.checkpointFlushWorker = &checkpointFlushWorker{ - input: make(chan *checkpointFlushTask, 16), - cp: syncer.checkpoint, - execError: &syncer.execError, - afterFlushFn: syncer.afterFlushCheckpoint, - addCountFunc: func(bool, string, opType, int64, *filter.Table) {}, + input: make(chan *checkpointFlushTask, 16), + cp: syncer.checkpoint, + execError: &syncer.execError, + afterFlushFn: syncer.afterFlushCheckpoint, + updateJobMetricsFn: func(bool, string, *job) {}, } syncer.handleJobFunc = syncer.addJobToMemory @@ -1161,28 +1162,28 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { []string{"CREATE TABLE IF NOT EXISTS `test_1`.`t_1` (`id` INT PRIMARY KEY,`name` VARCHAR(24))"}, nil, }, { - insert, + dml, []string{"REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(1), "a"}}, }, { - del, + dml, []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1"}, [][]interface{}{{int64(1)}}, }, { - update, + dml, []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1", "REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(2)}, {int64(1), "b"}}, }, { // start from this event, location passes safeModeExitLocation and safe mode should exit - insert, + dml, []string{"INSERT INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(1), "a"}}, }, { - del, + dml, []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1"}, [][]interface{}{{int64(1)}}, }, { - update, + dml, []string{"UPDATE `test_1`.`t_1` SET `id` = ?, `name` = ? WHERE `id` = ? LIMIT 1"}, [][]interface{}{{int64(1), "b", int64(2)}}, }, { @@ -1444,16 +1445,46 @@ type expectJob struct { args [][]interface{} } +var defaultDMLType = map[sqlmodel.RowChangeType]sqlmodel.DMLType{ + sqlmodel.RowChangeInsert: sqlmodel.DMLInsert, + sqlmodel.RowChangeUpdate: sqlmodel.DMLUpdate, + sqlmodel.RowChangeDelete: sqlmodel.DMLDelete, +} + func checkJobs(c *C, jobs []*job, expectJobs []*expectJob) { c.Assert(len(jobs), Equals, len(expectJobs), Commentf("jobs = %q", jobs)) for i, job := range jobs { c.Assert(job.tp, Equals, expectJobs[i].tp) + if job.tp == ddl { c.Assert(job.ddls, DeepEquals, expectJobs[i].sqlInJob) - } else if job.tp == insert || job.tp == update || job.tp == del { - sqls, args := job.dml.genSQL() - c.Assert(sqls, DeepEquals, expectJobs[i].sqlInJob) - c.Assert(args, DeepEquals, expectJobs[i].args) + continue + } + + if job.tp == dml { + if !job.safeMode { + sql, args := job.dml.GenSQL(defaultDMLType[job.dml.Type()]) + c.Assert([]string{sql}, DeepEquals, expectJobs[i].sqlInJob) + c.Assert([][]interface{}{args}, DeepEquals, expectJobs[i].args) + continue + } + + // safemode + switch job.dml.Type() { + case sqlmodel.RowChangeInsert: + sql, args := job.dml.GenSQL(sqlmodel.DMLReplace) + c.Assert([]string{sql}, DeepEquals, expectJobs[i].sqlInJob) + c.Assert([][]interface{}{args}, DeepEquals, expectJobs[i].args) + case sqlmodel.RowChangeUpdate: + sql, args := job.dml.GenSQL(sqlmodel.DMLDelete) + sql2, args2 := job.dml.GenSQL(sqlmodel.DMLReplace) + c.Assert([]string{sql, sql2}, DeepEquals, expectJobs[i].sqlInJob) + c.Assert([][]interface{}{args, args2}, DeepEquals, expectJobs[i].args) + case sqlmodel.RowChangeDelete: + sql, args := job.dml.GenSQL(sqlmodel.DMLDelete) + c.Assert([]string{sql}, DeepEquals, expectJobs[i].sqlInJob) + c.Assert([][]interface{}{args}, DeepEquals, expectJobs[i].args) + } } } } @@ -1463,11 +1494,21 @@ var testJobs struct { jobs []*job } +func newDummyJob(tp opType, targetTable *filter.Table, ddls ...string) *job { + return &job{ + tp: tp, + targetTable: targetTable, + ddls: ddls, + dml: &sqlmodel.RowChange{}, + } +} + func (s *Syncer) mockFinishJob(jobs []*expectJob) { for _, job := range jobs { switch job.tp { - case ddl, insert, update, del, flush: - s.addCount(true, "test", job.tp, 1, &filter.Table{}) + case ddl, dml, flush: + dummyJob := newDummyJob(job.tp, &filter.Table{}, job.sqlInJob...) + s.updateJobMetrics(true, "test", dummyJob) } } } @@ -1476,8 +1517,8 @@ func (s *Syncer) addJobToMemory(job *job) (bool, error) { log.L().Info("add job to memory", zap.Stringer("job", job)) switch job.tp { - case ddl, insert, update, del, flush: - s.addCount(false, "test", job.tp, 1, &filter.Table{}) + case ddl, dml, flush: + s.updateJobMetrics(false, "test", job) testJobs.Lock() testJobs.jobs = append(testJobs.jobs, job) testJobs.Unlock() @@ -1500,7 +1541,7 @@ func (s *Syncer) addJobToMemory(job *job) (bool, error) { } } s.resetShardingGroup(job.targetTable) - case insert, update, del: + case dml: for sourceSchema, tbs := range job.sourceTbls { if len(sourceSchema) == 0 { continue @@ -1527,11 +1568,11 @@ func (s *Syncer) setupMockCheckpoint(c *C, checkPointDBConn *sql.Conn, checkPoin s.checkpoint.(*RemoteCheckPoint).dbConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(checkPointDBConn, &retry.FiniteRetryStrategy{})} // mock syncer.flushCpWorker init s.checkpointFlushWorker = &checkpointFlushWorker{ - input: nil, - cp: s.checkpoint, - execError: &s.execError, - afterFlushFn: s.afterFlushCheckpoint, - addCountFunc: func(bool, string, opType, int64, *filter.Table) {}, + input: nil, + cp: s.checkpoint, + execError: &s.execError, + afterFlushFn: s.afterFlushCheckpoint, + updateJobMetricsFn: func(bool, string, *job) {}, } c.Assert(s.checkpoint.(*RemoteCheckPoint).prepare(tcontext.Background()), IsNil) // disable flush checkpoint periodically diff --git a/dm/tests/downstream_diff_index/run.sh b/dm/tests/downstream_diff_index/run.sh index eab1c4f87b9..99ecab92111 100755 --- a/dm/tests/downstream_diff_index/run.sh +++ b/dm/tests/downstream_diff_index/run.sh @@ -25,7 +25,7 @@ function run() { # worker will inject delete/update sql check inject_points=( - "github.com/pingcap/tiflow/dm/syncer/DownstreamTrackerWhereCheck=return()" + "github.com/pingcap/tiflow/pkg/sqlmodel/DownstreamTrackerWhereCheck=return()" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 101b9d4e7f8..1a9cb587b75 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -710,10 +710,10 @@ function DM_MULTIPLE_ROWS_CASE() { run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb} where a>100 and a<=200;" "count(1): 100" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml 30 - insertMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=insert\]' | wc -l) - replaceMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=replace\]' | wc -l) - updateMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op="insert on duplicate update"\]' | wc -l) - deleteMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=delete\]' | wc -l) + insertMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=DMLInsert\]' | wc -l) + replaceMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=DMLReplace\]' | wc -l) + updateMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=DMLInsertOnDuplicateUpdate\]' | wc -l) + deleteMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=DMLDelete\]' | wc -l) echo $insertMergeCnt $replaceMergeCnt $updateMergeCnt $deleteMergeCnt if [[ "$insertMergeCnt" -le 5 || "$updateMergeCnt" -le 5 || "$deleteMergeCnt" -le 5 || "$replaceMergeCnt" -le 5 ]]; then echo "merge dmls less than 5, insertMergeCnt: $insertMergeCnt, replaceMergeCnt: $replaceMergeCnt, updateMergeCnt: $updateMergeCnt, deleteMergeCnt: $deleteMergeCnt" diff --git a/pkg/sqlmodel/causality_test.go b/pkg/sqlmodel/causality_test.go index 4a49e95640e..998239168df 100644 --- a/pkg/sqlmodel/causality_test.go +++ b/pkg/sqlmodel/causality_test.go @@ -46,6 +46,38 @@ func TestCausalityKeys(t *testing.T) { []interface{}{3, 4, "adef"}, []string{"1.c.db.tb1", "a.c3.db.tb1", "3.c.db.tb1", "a.c3.db.tb1"}, }, + + // test not string key + { + "CREATE TABLE tb1 (a INT, b INT, UNIQUE KEY a(a))", + []interface{}{100, 200}, + nil, + []string{"100.a.db.tb1"}, + }, + + // test text + { + "CREATE TABLE tb1 (a INT, b TEXT, UNIQUE KEY b(b(3)))", + []interface{}{1, "1234"}, + nil, + []string{"123.b.db.tb1"}, + }, + + // test composite keys + { + "CREATE TABLE tb1 (a INT, b TEXT, UNIQUE KEY c2(a, b(3)))", + []interface{}{1, "1234"}, + nil, + []string{"1.a.123.b.db.tb1"}, + }, + + // test value is null + { + "CREATE TABLE tb1 (a INT, b TEXT, UNIQUE KEY c2(a, b(3)))", + []interface{}{1, nil}, + nil, + []string{"1.a.db.tb1"}, + }, } for _, ca := range cases { @@ -71,3 +103,103 @@ func TestCausalityKeysNoRace(t *testing.T) { } wg.Wait() } + +func TestGetCausalityString(t *testing.T) { + t.Parallel() + + source := &cdcmodel.TableName{Schema: "db", Table: "tbl"} + + testCases := []struct { + schema string + values []interface{} + keys []string + }{ + { + // test no keys will use full row data instead of table name + schema: `create table t1(a int)`, + values: []interface{}{10}, + keys: []string{"10.a.db.tbl"}, + }, + { + // one primary key + schema: `create table t2(a int primary key, b double)`, + values: []interface{}{60, 70.5}, + keys: []string{"60.a.db.tbl"}, + }, + { + // one unique key + schema: `create table t3(a int unique, b double)`, + values: []interface{}{60, 70.5}, + keys: []string{"60.a.db.tbl"}, + }, + { + // one ordinary key + schema: `create table t4(a int, b double, key(b))`, + values: []interface{}{60, 70.5}, + keys: []string{"60.a.70.5.b.db.tbl"}, + }, + { + // multiple keys + schema: `create table t5(a int, b text, c int, key(a), key(b(3)))`, + values: []interface{}{13, "abcdef", 15}, + keys: []string{"13.a.abcdef.b.15.c.db.tbl"}, + }, + { + // multiple keys with primary key + schema: `create table t6(a int primary key, b varchar(16) unique)`, + values: []interface{}{16, "xyz"}, + keys: []string{"16.a.db.tbl", "xyz.b.db.tbl"}, + }, + { + // non-integer primary key + schema: `create table t65(a int unique, b varchar(16) primary key)`, + values: []interface{}{16, "xyz"}, + keys: []string{"xyz.b.db.tbl", "16.a.db.tbl"}, + }, + { + // primary key of multiple columns + schema: `create table t7(a int, b int, primary key(a, b))`, + values: []interface{}{59, 69}, + keys: []string{"59.a.69.b.db.tbl"}, + }, + { + // ordinary key of multiple columns + schema: `create table t75(a int, b int, c int, key(a, b), key(c, b))`, + values: []interface{}{48, 58, 68}, + keys: []string{"48.a.58.b.68.c.db.tbl"}, + }, + { + // so many keys + schema: ` + create table t8( + a int, b int, c int, + primary key(a, b), + unique key(b, c), + key(a, b, c), + unique key(c, a) + ) + `, + values: []interface{}{27, 37, 47}, + keys: []string{"27.a.37.b.db.tbl", "37.b.47.c.db.tbl", "47.c.27.a.db.tbl"}, + }, + { + // `null` for unique key + schema: ` + create table t8( + a int, b int default null, + primary key(a), + unique key(b) + ) + `, + values: []interface{}{17, nil}, + keys: []string{"17.a.db.tbl"}, + }, + } + + for _, ca := range testCases { + ti := mockTableInfo(t, ca.schema) + change := NewRowChange(source, nil, nil, ca.values, ti, nil, nil) + change.lazyInitIdentityInfo() + require.Equal(t, ca.keys, change.getCausalityString(ca.values)) + } +} diff --git a/pkg/sqlmodel/reduce.go b/pkg/sqlmodel/reduce.go index 6e146e2d54c..807d8af64a8 100644 --- a/pkg/sqlmodel/reduce.go +++ b/pkg/sqlmodel/reduce.go @@ -120,11 +120,11 @@ func (r *RowChange) Reduce(preRowChange *RowChange) { r.calculateType() } -// Split will split current RowChangeUpdate into two RowChangeDelete and +// SplitUpdate will split current RowChangeUpdate into two RowChangeDelete and // RowChangeInsert one. The behaviour is undefined for other types of RowChange. -func (r *RowChange) Split() (*RowChange, *RowChange) { +func (r *RowChange) SplitUpdate() (*RowChange, *RowChange) { if r.tp != RowChangeUpdate { - log.L().DPanic("Split should only be called on RowChangeUpdate", + log.L().DPanic("SplitUpdate should only be called on RowChangeUpdate", zap.Stringer("rowChange", r)) return nil, nil } diff --git a/pkg/sqlmodel/reduce_test.go b/pkg/sqlmodel/reduce_test.go index 876e2089252..8592b91b48f 100644 --- a/pkg/sqlmodel/reduce_test.go +++ b/pkg/sqlmodel/reduce_test.go @@ -53,7 +53,7 @@ func TestSplit(t *testing.T) { change := NewRowChange(source, nil, []interface{}{1, 2}, []interface{}{3, 4}, sourceTI1, nil, nil) require.True(t, change.IsIdentityUpdated()) - del, ins := change.Split() + del, ins := change.SplitUpdate() delIDKey := del.IdentityKey() require.NotZero(t, delIDKey) insIDKey := ins.IdentityKey() diff --git a/pkg/sqlmodel/row_change.go b/pkg/sqlmodel/row_change.go index a4a3cbfd8ca..7c29ce7dafa 100644 --- a/pkg/sqlmodel/row_change.go +++ b/pkg/sqlmodel/row_change.go @@ -375,3 +375,23 @@ func (r *RowChange) GenSQL(tp DMLType) (string, []interface{}) { zap.Stringer("DMLType", tp)) return "", nil } + +// GetPreValues is only used in tests. +func (r *RowChange) GetPreValues() []interface{} { + return r.preValues +} + +// GetPostValues is only used in tests. +func (r *RowChange) GetPostValues() []interface{} { + return r.postValues +} + +// GetSourceTable returns TableName of the source table. +func (r *RowChange) GetSourceTable() *cdcmodel.TableName { + return r.sourceTable +} + +// GetTargetTable returns TableName of the target table. +func (r *RowChange) GetTargetTable() *cdcmodel.TableName { + return r.targetTable +}