From 4e8689eed7be035e0283f478905dc9207b3f87d0 Mon Sep 17 00:00:00 2001 From: colin Date: Thu, 13 Aug 2020 00:40:13 +0800 Subject: [PATCH 01/35] create dev branch --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index dfb5f29e348..15da4afa526 100644 --- a/README.md +++ b/README.md @@ -68,3 +68,5 @@ for details on submitting patches and the contribution workflow. ## License TiCDC is under the Apache 2.0 license. See the [LICENSE](./LICENSE) file for details. + +## Branch For dev From 88318c3dda51f35ab5ddeeb8558c02c21b74e183 Mon Sep 17 00:00:00 2001 From: colin Date: Wed, 19 Aug 2020 01:02:40 +0800 Subject: [PATCH 02/35] add create sync point logic --- cdc/changefeed.go | 54 +++++++++++++++++++++++++++++++++++++++-------- cdc/owner.go | 4 ++++ 2 files changed, 49 insertions(+), 9 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 7239857b0d4..4c87fc2231d 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -34,6 +34,8 @@ import ( "go.uber.org/zap" ) +const syncInterval = time.Minute * 10 //同步间隔为10分钟 todo 需要参数化 + type tableIDMap = map[model.TableID]struct{} // OwnerDDLHandler defines the ddl handler for Owner @@ -77,14 +79,16 @@ type changeFeed struct { info *model.ChangeFeedInfo status *model.ChangeFeedStatus - schema *entry.SingleSchemaSnapshot - ddlState model.ChangeFeedDDLState - targetTs uint64 - taskStatus model.ProcessorsInfos - taskPositions map[model.CaptureID]*model.TaskPosition - filter *filter.Filter - sink sink.Sink - scheduler scheduler.Scheduler + schema *entry.SingleSchemaSnapshot + ddlState model.ChangeFeedDDLState + targetTs uint64 + updateResolvedTs bool + startTimer chan bool + taskStatus model.ProcessorsInfos + taskPositions map[model.CaptureID]*model.TaskPosition + filter *filter.Filter + sink sink.Sink + scheduler scheduler.Scheduler cyclicEnabled bool @@ -711,6 +715,18 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { } } + if c.status.ResolvedTs == c.status.CheckpointTs || c.status.ResolvedTs == 0 { + log.Debug("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs)) + //c.syncPointTs = c.targetTs //恢复syncPointTs,使得ResoledTs可以继续推进 + c.updateResolvedTs = true + + //todo 实现向下游同步记录同步点 + } + + /*if minResolvedTs > c.syncPointTs { + minResolvedTs = c.syncPointTs + }*/ + if len(c.taskPositions) < len(c.taskStatus) { log.Debug("skip update resolved ts", zap.Int("taskPositions", len(c.taskPositions)), @@ -802,9 +818,15 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { var tsUpdated bool - if minResolvedTs > c.status.ResolvedTs { + if c.updateResolvedTs && minResolvedTs > c.status.ResolvedTs { + prevResolvedTs := c.status.ResolvedTs c.status.ResolvedTs = minResolvedTs tsUpdated = true + if prevResolvedTs == c.status.CheckpointTs || prevResolvedTs == 0 { + //到达sync point + //todo 需要重新开始启动计时 + c.startTimer <- true + } } if minCheckpointTs > c.status.CheckpointTs { @@ -845,3 +867,17 @@ func (c *changeFeed) pullDDLJob() error { } return nil } + +// startSyncPeriod start a timer for every changefeed to create sync point by time +func (c *changeFeed) startSyncPeriod() { + //c.startTimer <- true + go func() { + for { + select { + case <-c.startTimer: + time.Sleep(syncInterval) + c.updateResolvedTs = false + } + } + }() +} diff --git a/cdc/owner.go b/cdc/owner.go index 8766168a9b0..53b25608c34 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -344,6 +344,8 @@ func (o *Owner) newChangeFeed( ddlState: model.ChangeFeedSyncDML, ddlExecutedTs: checkpointTs, targetTs: info.GetTargetTs(), + updateResolvedTs: true, + startTimer: make(chan bool), taskStatus: processorsInfos, taskPositions: taskPositions, etcdCli: o.etcdClient, @@ -494,6 +496,8 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { continue } o.changeFeeds[changeFeedID] = newCf + //start SyncPeriod for create the sync point + newCf.startSyncPeriod() delete(o.stoppedFeeds, changeFeedID) } o.adminJobsLock.Lock() From e4cadbd2cb32d29c49feb6b04d81631e624574fb Mon Sep 17 00:00:00 2001 From: colin Date: Thu, 20 Aug 2020 19:40:05 +0800 Subject: [PATCH 03/35] add sync table logic --- cdc/changefeed.go | 60 ++++++++++++++++++++++++ cdc/owner.go | 117 ++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 177 insertions(+) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 4c87fc2231d..51caeed3aab 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -15,6 +15,7 @@ package cdc import ( "context" + "database/sql" "fmt" "math" "time" @@ -84,6 +85,7 @@ type changeFeed struct { targetTs uint64 updateResolvedTs bool startTimer chan bool + syncDB *sql.DB taskStatus model.ProcessorsInfos taskPositions map[model.CaptureID]*model.TaskPosition filter *filter.Filter @@ -721,6 +723,55 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { c.updateResolvedTs = true //todo 实现向下游同步记录同步点 + //rows := make([]*model.RowChangedEvent, 0, defaultSyncResolvedBatch) + //row := new(model.RowChangedEvent) + /*row := &model.RowChangedEvent{ + StartTs: 0, //好像没问题 + CommitTs: c.status.CheckpointTs, + RowID: 0, //注意设置一下 好像没问题 + TableInfoVersion: 0, // 不懂 好像没问题 + Table: &model.TableName{ + Schema: "TiCDC", + Table: "syncpoint", + Partition: 0, //不懂 好像没问题 + }, + IndieMarkCol: "master-ts", //待定 + Delete: false, + Columns: make([]*model.Column, 1), //待定 + PreColumns: nil, //待定 + IndexColumns: nil, //待定 + // FIXME(leoppor): Correctness of conflict detection with old values + //Keys: genMultipleKeys(tableInfo, preCols, cols, quotes.QuoteSchema(schemaName, tableName)), + Keys: "test", + } + row.Columns = append(row.Columns, &model.Column{Name: "master-ts", Type: mysql.TypeVarchar, Flag: model.PrimaryKeyFlag, Value: c.status.CheckpointTs}) + row.Columns = append(row.Columns, &model.Column{Name: "slave-ts", Type: mysql.TypeVarchar, Flag: model.NullableFlag, Value: "select @@tidb_current_ts"}) + rows = append(rows, row) + /*row.StartTs = 0 //todo 不知道怎么设置 + row.CommitTs = c.status.CheckpointTs + row.RowID = 0 //todo 不知道怎么设置 + row.Table = new(model.TableName) + row.Table.Schema = "TiCDC" + row.Table.Table = "syncpoint" + row.Table.Partition = 0 */ + //c.sink.FlushRowChangedEvents(ctx, c.status.CheckpointTs) + /*c.sink.EmitRowChangedEvents(ctx, rows...) + c.sink.FlushRowChangedEvents(ctx, c.status.CheckpointTs)*/ + tx, err := c.syncDB.BeginTx(ctx, nil) + if err != nil { + log.Info("sync table: begin Tx fail") + return err + } + row, err := tx.Query("select @@tidb_current_ts") + if err != nil { + log.Info("sync table: get tidb_current_ts err") + return err + } + var slaveTs string + row.Scan(&slaveTs) + tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, slaveTs) + tx.Commit() //TODO 处理错误 + } /*if minResolvedTs > c.syncPointTs { @@ -881,3 +932,12 @@ func (c *changeFeed) startSyncPeriod() { } }() } + +//createSynctable create a sync table to record the +func (c *changeFeed) createSynctable() { + database := "TiCDC" + c.syncDB.Exec("CREATE DATABASE IF NOT EXISTS " + database) + c.syncDB.Exec("USE " + database) + c.syncDB.Exec("CREATE TABLE IF NOT EXISTS syncpoint ( master_ts varchar(18),slave_ts varchar(18),PRIMARY KEY ( `master_ts` ) )") + // todo err 处理 +} diff --git a/cdc/owner.go b/cdc/owner.go index 53b25608c34..9cb8f061cd4 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -15,14 +15,17 @@ package cdc import ( "context" + "database/sql" "fmt" "io" "math" + "net/url" "os" "strings" "sync" "time" + dmysql "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" @@ -326,6 +329,72 @@ func (o *Owner) newChangeFeed( log.Error("error on running owner", zap.Error(err)) } + //TODO 后期需要打包一下 + var syncDB *sql.DB + // parse sinkURI as a URI + sinkURI, err := url.Parse(info.SinkURI) + if err != nil { + return nil, errors.Annotatef(err, "parse sinkURI failed") + } + if sinkURI == nil { + return nil, errors.New("fail to open MySQL sink, empty URL") + } + scheme := strings.ToLower(sinkURI.Scheme) + if scheme != "mysql" && scheme != "tidb" { + return nil, errors.New("can create mysql sink with unsupported scheme") + } + var tlsParam string + if sinkURI.Query().Get("ssl-ca") != "" { + credential := security.Credential{ + CAPath: sinkURI.Query().Get("ssl-ca"), + CertPath: sinkURI.Query().Get("ssl-cert"), + KeyPath: sinkURI.Query().Get("ssl-key"), + } + tlsCfg, err := credential.ToTLSConfig() + if err != nil { + return nil, errors.Annotate(err, "fail to open MySQL connection") + } + name := "cdc_mysql_tls" + "syncpoint" + id + err = dmysql.RegisterTLSConfig(name, tlsCfg) + if err != nil { + return nil, errors.Annotate(err, "fail to open MySQL connection") + } + tlsParam = "?tls=" + name + } + + // dsn format of the driver: + // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] + username := sinkURI.User.Username() + password, _ := sinkURI.User.Password() + port := sinkURI.Port() + if username == "" { + username = "root" + } + if port == "" { + port = "4000" + } + + dsnStr := fmt.Sprintf("%s:%s@tcp(%s:%s)/%s", username, password, sinkURI.Hostname(), port, tlsParam) + dsn, err := dmysql.ParseDSN(dsnStr) + if err != nil { + return nil, errors.Trace(err) + } + dsnStr, err = configureSinkURI(ctx, dsn, util.TimezoneFromCtx(ctx)) + if err != nil { + return nil, errors.Trace(err) + } + syncDB, err = sql.Open("mysql", dsnStr) + if err != nil { + return nil, errors.Annotate(err, "Open database connection failed") + } + err = syncDB.PingContext(ctx) + if err != nil { + return nil, errors.Annotatef(err, "fail to open MySQL connection") + } + + log.Info("Start mysql syncpoint sink") + //需要打包 end + cf := &changeFeed{ info: info, id: id, @@ -346,6 +415,7 @@ func (o *Owner) newChangeFeed( targetTs: info.GetTargetTs(), updateResolvedTs: true, startTimer: make(chan bool), + syncDB: syncDB, taskStatus: processorsInfos, taskPositions: taskPositions, etcdCli: o.etcdClient, @@ -496,6 +566,8 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { continue } o.changeFeeds[changeFeedID] = newCf + //create the sync table + newCf.createSynctable() //start SyncPeriod for create the sync point newCf.startSyncPeriod() delete(o.stoppedFeeds, changeFeedID) @@ -1255,3 +1327,48 @@ func (o *Owner) startCaptureWatcher(ctx context.Context) { } }() } + +func configureSinkURI(ctx context.Context, dsnCfg *dmysql.Config, tz *time.Location) (string, error) { + if dsnCfg.Params == nil { + dsnCfg.Params = make(map[string]string, 1) + } + dsnCfg.DBName = "" + dsnCfg.InterpolateParams = true + dsnCfg.MultiStatements = true + dsnCfg.Params["time_zone"] = fmt.Sprintf(`"%s"`, tz.String()) + + testDB, err := sql.Open("mysql", dsnCfg.FormatDSN()) + if err != nil { + return "", errors.Annotate(err, "fail to open MySQL connection when configuring sink") + } + defer testDB.Close() + log.Debug("Opened connection to configure some tidb special parameters") + + var variableName string + var autoRandomInsertEnabled string + queryStr := "show session variables like 'allow_auto_random_explicit_insert';" + err = testDB.QueryRowContext(ctx, queryStr).Scan(&variableName, &autoRandomInsertEnabled) + if err != nil && err != sql.ErrNoRows { + return "", errors.Annotate(err, "fail to query sink for support of auto-random") + } + if err == nil && (autoRandomInsertEnabled == "off" || autoRandomInsertEnabled == "0") { + dsnCfg.Params["allow_auto_random_explicit_insert"] = "1" + log.Debug("Set allow_auto_random_explicit_insert to 1") + } + + var txnMode string + queryStr = "show session variables like 'tidb_txn_mode';" + err = testDB.QueryRowContext(ctx, queryStr).Scan(&variableName, &txnMode) + if err != nil && err != sql.ErrNoRows { + return "", errors.Annotate(err, "fail to query sink for txn mode") + } + /*if err == nil { + dsnCfg.Params["tidb_txn_mode"] = params.tidbTxnMode + }*/ + + dsnClone := dsnCfg.Clone() + dsnClone.Passwd = "******" + log.Info("sink uri is configured", zap.String("format dsn", dsnClone.FormatDSN())) + + return dsnCfg.FormatDSN(), nil +} From f26eead79c2cbe1d6a00ebc8faf62020caf9bb12 Mon Sep 17 00:00:00 2001 From: colin Date: Tue, 1 Sep 2020 09:23:22 +0800 Subject: [PATCH 04/35] add proxy for docker --- Dockerfile | 1 + cdc/changefeed.go | 21 +++++++++++---------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/Dockerfile b/Dockerfile index b04ec56945d..b76c609b9b9 100644 --- a/Dockerfile +++ b/Dockerfile @@ -2,6 +2,7 @@ FROM golang:1.14-alpine as builder RUN apk add --no-cache git make bash WORKDIR /go/src/github.com/pingcap/ticdc COPY . . +RUN go env -w GO111MODULE=on && go env -w GOPROXY=https://goproxy.cn,direct RUN make FROM alpine:3.12 diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 51caeed3aab..9aa29bc5b80 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -762,14 +762,15 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { log.Info("sync table: begin Tx fail") return err } - row, err := tx.Query("select @@tidb_current_ts") + /*row, err := tx.Query("select @@tidb_current_ts") if err != nil { log.Info("sync table: get tidb_current_ts err") return err } var slaveTs string row.Scan(&slaveTs) - tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, slaveTs) + tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, slaveTs)*/ + tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, 0) tx.Commit() //TODO 处理错误 } @@ -870,14 +871,14 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { var tsUpdated bool if c.updateResolvedTs && minResolvedTs > c.status.ResolvedTs { - prevResolvedTs := c.status.ResolvedTs + //prevResolvedTs := c.status.ResolvedTs c.status.ResolvedTs = minResolvedTs tsUpdated = true - if prevResolvedTs == c.status.CheckpointTs || prevResolvedTs == 0 { + /*if prevResolvedTs == c.status.CheckpointTs || prevResolvedTs == 0 { //到达sync point //todo 需要重新开始启动计时 c.startTimer <- true - } + }*/ } if minCheckpointTs > c.status.CheckpointTs { @@ -924,11 +925,11 @@ func (c *changeFeed) startSyncPeriod() { //c.startTimer <- true go func() { for { - select { - case <-c.startTimer: - time.Sleep(syncInterval) - c.updateResolvedTs = false - } + //select { + //case <-c.startTimer: + time.Sleep(syncInterval) + c.updateResolvedTs = false + //} } }() } From 3fd1b3dbbc2e4102126881b537f81a361a93910d Mon Sep 17 00:00:00 2001 From: colin Date: Tue, 1 Sep 2020 22:08:50 +0800 Subject: [PATCH 05/35] [9.1] fix the logic --- cdc/changefeed.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 9aa29bc5b80..fa9e585420a 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -717,7 +717,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { } } - if c.status.ResolvedTs == c.status.CheckpointTs || c.status.ResolvedTs == 0 { + if (c.status.ResolvedTs == c.status.CheckpointTs && (c.updateResolvedTs == false || c.status.ResolvedTs == c.ddlResolvedTs)) || c.status.ResolvedTs == 0 { log.Debug("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs)) //c.syncPointTs = c.targetTs //恢复syncPointTs,使得ResoledTs可以继续推进 c.updateResolvedTs = true @@ -762,17 +762,17 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { log.Info("sync table: begin Tx fail") return err } - /*row, err := tx.Query("select @@tidb_current_ts") + row := tx.QueryRow("select @@tidb_current_ts") + var slaveTs string + err = row.Scan(&slaveTs) if err != nil { log.Info("sync table: get tidb_current_ts err") + tx.Rollback() return err } - var slaveTs string - row.Scan(&slaveTs) - tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, slaveTs)*/ - tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, 0) + tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, slaveTs) + //tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, 0) tx.Commit() //TODO 处理错误 - } /*if minResolvedTs > c.syncPointTs { From 97a161599350d398cd2ef14aaef6471df0b07a3a Mon Sep 17 00:00:00 2001 From: colin Date: Thu, 3 Sep 2020 16:25:56 +0800 Subject: [PATCH 06/35] record the sync point base on timer(the ddl and table migration sync point has not add yet) --- cdc/changefeed.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index fa9e585420a..0ced525d8d7 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -717,9 +717,11 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { } } - if (c.status.ResolvedTs == c.status.CheckpointTs && (c.updateResolvedTs == false || c.status.ResolvedTs == c.ddlResolvedTs)) || c.status.ResolvedTs == 0 { + //if (c.status.ResolvedTs == c.status.CheckpointTs && (c.updateResolvedTs == false || c.status.ResolvedTs == c.ddlResolvedTs)) || c.status.ResolvedTs == 0 { + if (c.status.ResolvedTs == c.status.CheckpointTs && c.updateResolvedTs == false) || c.status.ResolvedTs == 0 { log.Debug("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs)) //c.syncPointTs = c.targetTs //恢复syncPointTs,使得ResoledTs可以继续推进 + log.Info("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs)) c.updateResolvedTs = true //todo 实现向下游同步记录同步点 From 94ec1d4f48edda524586768e76f4c221dd270459 Mon Sep 17 00:00:00 2001 From: colin Date: Sat, 5 Sep 2020 22:58:27 +0800 Subject: [PATCH 07/35] add ddl syncpoint record --- cdc/changefeed.go | 87 +++++++++++++++++------------------------------ cdc/owner.go | 1 + 2 files changed, 32 insertions(+), 56 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 0ced525d8d7..1a7354657c0 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -83,6 +83,7 @@ type changeFeed struct { schema *entry.SingleSchemaSnapshot ddlState model.ChangeFeedDDLState targetTs uint64 + ddlTs uint64 updateResolvedTs bool startTimer chan bool syncDB *sql.DB @@ -721,65 +722,16 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { if (c.status.ResolvedTs == c.status.CheckpointTs && c.updateResolvedTs == false) || c.status.ResolvedTs == 0 { log.Debug("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs)) //c.syncPointTs = c.targetTs //恢复syncPointTs,使得ResoledTs可以继续推进 - log.Info("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs)) + log.Info("achive the sync point with timer", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) c.updateResolvedTs = true - - //todo 实现向下游同步记录同步点 - //rows := make([]*model.RowChangedEvent, 0, defaultSyncResolvedBatch) - //row := new(model.RowChangedEvent) - /*row := &model.RowChangedEvent{ - StartTs: 0, //好像没问题 - CommitTs: c.status.CheckpointTs, - RowID: 0, //注意设置一下 好像没问题 - TableInfoVersion: 0, // 不懂 好像没问题 - Table: &model.TableName{ - Schema: "TiCDC", - Table: "syncpoint", - Partition: 0, //不懂 好像没问题 - }, - IndieMarkCol: "master-ts", //待定 - Delete: false, - Columns: make([]*model.Column, 1), //待定 - PreColumns: nil, //待定 - IndexColumns: nil, //待定 - // FIXME(leoppor): Correctness of conflict detection with old values - //Keys: genMultipleKeys(tableInfo, preCols, cols, quotes.QuoteSchema(schemaName, tableName)), - Keys: "test", - } - row.Columns = append(row.Columns, &model.Column{Name: "master-ts", Type: mysql.TypeVarchar, Flag: model.PrimaryKeyFlag, Value: c.status.CheckpointTs}) - row.Columns = append(row.Columns, &model.Column{Name: "slave-ts", Type: mysql.TypeVarchar, Flag: model.NullableFlag, Value: "select @@tidb_current_ts"}) - rows = append(rows, row) - /*row.StartTs = 0 //todo 不知道怎么设置 - row.CommitTs = c.status.CheckpointTs - row.RowID = 0 //todo 不知道怎么设置 - row.Table = new(model.TableName) - row.Table.Schema = "TiCDC" - row.Table.Table = "syncpoint" - row.Table.Partition = 0 */ - //c.sink.FlushRowChangedEvents(ctx, c.status.CheckpointTs) - /*c.sink.EmitRowChangedEvents(ctx, rows...) - c.sink.FlushRowChangedEvents(ctx, c.status.CheckpointTs)*/ - tx, err := c.syncDB.BeginTx(ctx, nil) - if err != nil { - log.Info("sync table: begin Tx fail") - return err - } - row := tx.QueryRow("select @@tidb_current_ts") - var slaveTs string - err = row.Scan(&slaveTs) - if err != nil { - log.Info("sync table: get tidb_current_ts err") - tx.Rollback() - return err - } - tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, slaveTs) - //tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, 0) - tx.Commit() //TODO 处理错误 + c.sinkSyncpoint(ctx) } - /*if minResolvedTs > c.syncPointTs { - minResolvedTs = c.syncPointTs - }*/ + if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs { + log.Info("achive the sync point with ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) + c.sinkSyncpoint(ctx) + c.ddlTs = 0 + } if len(c.taskPositions) < len(c.taskStatus) { log.Debug("skip update resolved ts", @@ -851,6 +803,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { if minResolvedTs > c.ddlResolvedTs { minResolvedTs = c.ddlResolvedTs + //c.ddlTs = c.ddlResolvedTs } } @@ -862,6 +815,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { if len(c.ddlJobHistory) > 0 && minResolvedTs >= c.ddlJobHistory[0].BinlogInfo.FinishedTS { minResolvedTs = c.ddlJobHistory[0].BinlogInfo.FinishedTS c.ddlState = model.ChangeFeedWaitToExecDDL + c.ddlTs = minResolvedTs } // if downstream sink is the MQ sink, the MQ sink do not promise that checkpoint is less than globalResolvedTs @@ -944,3 +898,24 @@ func (c *changeFeed) createSynctable() { c.syncDB.Exec("CREATE TABLE IF NOT EXISTS syncpoint ( master_ts varchar(18),slave_ts varchar(18),PRIMARY KEY ( `master_ts` ) )") // todo err 处理 } + +//sinkSyncpoint record the syncpoint(a map with ts) in downstream db +func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { + tx, err := c.syncDB.BeginTx(ctx, nil) + if err != nil { + log.Info("sync table: begin Tx fail") + return err + } + row := tx.QueryRow("select @@tidb_current_ts") + var slaveTs string + err = row.Scan(&slaveTs) + if err != nil { + log.Info("sync table: get tidb_current_ts err") + tx.Rollback() + return err + } + tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, slaveTs) + //tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, 0) + tx.Commit() //TODO 处理错误 + return nil +} diff --git a/cdc/owner.go b/cdc/owner.go index 9cb8f061cd4..e187353e7eb 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -413,6 +413,7 @@ func (o *Owner) newChangeFeed( ddlState: model.ChangeFeedSyncDML, ddlExecutedTs: checkpointTs, targetTs: info.GetTargetTs(), + ddlTs: 0, updateResolvedTs: true, startTimer: make(chan bool), syncDB: syncDB, From 6abad54d45ec12b21740f5ddb2e1ea18217c0517 Mon Sep 17 00:00:00 2001 From: colin Date: Mon, 7 Sep 2020 20:04:05 +0800 Subject: [PATCH 08/35] a liitle change for createSyncpointSink() --- cdc/owner.go | 136 +++++++++++++++++++++++++++------------------------ 1 file changed, 72 insertions(+), 64 deletions(-) diff --git a/cdc/owner.go b/cdc/owner.go index e187353e7eb..078654691b9 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -329,72 +329,11 @@ func (o *Owner) newChangeFeed( log.Error("error on running owner", zap.Error(err)) } - //TODO 后期需要打包一下 - var syncDB *sql.DB - // parse sinkURI as a URI - sinkURI, err := url.Parse(info.SinkURI) + syncDB, err := o.createSyncpointSink(ctx, info, id) if err != nil { - return nil, errors.Annotatef(err, "parse sinkURI failed") - } - if sinkURI == nil { - return nil, errors.New("fail to open MySQL sink, empty URL") - } - scheme := strings.ToLower(sinkURI.Scheme) - if scheme != "mysql" && scheme != "tidb" { - return nil, errors.New("can create mysql sink with unsupported scheme") - } - var tlsParam string - if sinkURI.Query().Get("ssl-ca") != "" { - credential := security.Credential{ - CAPath: sinkURI.Query().Get("ssl-ca"), - CertPath: sinkURI.Query().Get("ssl-cert"), - KeyPath: sinkURI.Query().Get("ssl-key"), - } - tlsCfg, err := credential.ToTLSConfig() - if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") - } - name := "cdc_mysql_tls" + "syncpoint" + id - err = dmysql.RegisterTLSConfig(name, tlsCfg) - if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") - } - tlsParam = "?tls=" + name - } - - // dsn format of the driver: - // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] - username := sinkURI.User.Username() - password, _ := sinkURI.User.Password() - port := sinkURI.Port() - if username == "" { - username = "root" - } - if port == "" { - port = "4000" - } - - dsnStr := fmt.Sprintf("%s:%s@tcp(%s:%s)/%s", username, password, sinkURI.Hostname(), port, tlsParam) - dsn, err := dmysql.ParseDSN(dsnStr) - if err != nil { - return nil, errors.Trace(err) - } - dsnStr, err = configureSinkURI(ctx, dsn, util.TimezoneFromCtx(ctx)) - if err != nil { - return nil, errors.Trace(err) - } - syncDB, err = sql.Open("mysql", dsnStr) - if err != nil { - return nil, errors.Annotate(err, "Open database connection failed") - } - err = syncDB.PingContext(ctx) - if err != nil { - return nil, errors.Annotatef(err, "fail to open MySQL connection") + log.Error("error on running owner", zap.Error(err)) } - log.Info("Start mysql syncpoint sink") - //需要打包 end - cf := &changeFeed{ info: info, id: id, @@ -1329,7 +1268,7 @@ func (o *Owner) startCaptureWatcher(ctx context.Context) { }() } -func configureSinkURI(ctx context.Context, dsnCfg *dmysql.Config, tz *time.Location) (string, error) { +func (o *Owner) configureSinkURI(ctx context.Context, dsnCfg *dmysql.Config, tz *time.Location) (string, error) { if dsnCfg.Params == nil { dsnCfg.Params = make(map[string]string, 1) } @@ -1373,3 +1312,72 @@ func configureSinkURI(ctx context.Context, dsnCfg *dmysql.Config, tz *time.Locat return dsnCfg.FormatDSN(), nil } + +//createSyncpointSink create a sink to record the syncpoint map in downstream DB for every changefeed +func (o *Owner) createSyncpointSink(ctx context.Context, info *model.ChangeFeedInfo, id string) (*sql.DB, error) { + var syncDB *sql.DB + // parse sinkURI as a URI + sinkURI, err := url.Parse(info.SinkURI) + if err != nil { + return nil, errors.Annotatef(err, "parse sinkURI failed") + } + if sinkURI == nil { + return nil, errors.New("fail to open MySQL sink, empty URL") + } + scheme := strings.ToLower(sinkURI.Scheme) + if scheme != "mysql" && scheme != "tidb" { + return nil, errors.New("can create mysql sink with unsupported scheme") + } + var tlsParam string + if sinkURI.Query().Get("ssl-ca") != "" { + credential := security.Credential{ + CAPath: sinkURI.Query().Get("ssl-ca"), + CertPath: sinkURI.Query().Get("ssl-cert"), + KeyPath: sinkURI.Query().Get("ssl-key"), + } + tlsCfg, err := credential.ToTLSConfig() + if err != nil { + return nil, errors.Annotate(err, "fail to open MySQL connection") + } + name := "cdc_mysql_tls" + "syncpoint" + id + err = dmysql.RegisterTLSConfig(name, tlsCfg) + if err != nil { + return nil, errors.Annotate(err, "fail to open MySQL connection") + } + tlsParam = "?tls=" + name + } + + // dsn format of the driver: + // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] + username := sinkURI.User.Username() + password, _ := sinkURI.User.Password() + port := sinkURI.Port() + if username == "" { + username = "root" + } + if port == "" { + port = "4000" + } + + dsnStr := fmt.Sprintf("%s:%s@tcp(%s:%s)/%s", username, password, sinkURI.Hostname(), port, tlsParam) + dsn, err := dmysql.ParseDSN(dsnStr) + if err != nil { + return nil, errors.Trace(err) + } + dsnStr, err = o.configureSinkURI(ctx, dsn, util.TimezoneFromCtx(ctx)) + if err != nil { + return nil, errors.Trace(err) + } + syncDB, err = sql.Open("mysql", dsnStr) + if err != nil { + return nil, errors.Annotate(err, "Open database connection failed") + } + err = syncDB.PingContext(ctx) + if err != nil { + return nil, errors.Annotatef(err, "fail to open MySQL connection") + } + + log.Info("Start mysql syncpoint sink") + + return syncDB, nil +} From c13ccc3f8041108b2ee5c6fe46ba0e12f38bafed Mon Sep 17 00:00:00 2001 From: colin Date: Wed, 9 Sep 2020 01:26:50 +0800 Subject: [PATCH 09/35] 9.9 fix for syncpointsinl logic --- cdc/changefeed.go | 27 ++++---- cdc/owner.go | 158 +++++++++++++--------------------------------- cdc/sink/mysql.go | 76 ++++++++++++++++++++++ 3 files changed, 133 insertions(+), 128 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 1a7354657c0..4c60b1d57d3 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -35,7 +35,7 @@ import ( "go.uber.org/zap" ) -const syncInterval = time.Minute * 10 //同步间隔为10分钟 todo 需要参数化 +const syncInterval = time.Minute * 10 //Interval for set syncpoint, todo: parameterization type tableIDMap = map[model.TableID]struct{} @@ -719,7 +719,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { } //if (c.status.ResolvedTs == c.status.CheckpointTs && (c.updateResolvedTs == false || c.status.ResolvedTs == c.ddlResolvedTs)) || c.status.ResolvedTs == 0 { - if (c.status.ResolvedTs == c.status.CheckpointTs && c.updateResolvedTs == false) || c.status.ResolvedTs == 0 { + if (c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs) || c.status.ResolvedTs == 0 { log.Debug("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs)) //c.syncPointTs = c.targetTs //恢复syncPointTs,使得ResoledTs可以继续推进 log.Info("achive the sync point with timer", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) @@ -877,17 +877,18 @@ func (c *changeFeed) pullDDLJob() error { } // startSyncPeriod start a timer for every changefeed to create sync point by time -func (c *changeFeed) startSyncPeriod() { - //c.startTimer <- true - go func() { +func (c *changeFeed) startSyncPeriod(ctx context.Context) { + go func(ctx context.Context) { + ticker := time.NewTicker(syncInterval) for { - //select { - //case <-c.startTimer: - time.Sleep(syncInterval) - c.updateResolvedTs = false - //} + select { + case <-ctx.Done(): + return + case <-ticker.C: + c.updateResolvedTs = false + } } - }() + }(ctx) } //createSynctable create a sync table to record the @@ -895,7 +896,7 @@ func (c *changeFeed) createSynctable() { database := "TiCDC" c.syncDB.Exec("CREATE DATABASE IF NOT EXISTS " + database) c.syncDB.Exec("USE " + database) - c.syncDB.Exec("CREATE TABLE IF NOT EXISTS syncpoint ( master_ts varchar(18),slave_ts varchar(18),PRIMARY KEY ( `master_ts` ) )") + c.syncDB.Exec("CREATE TABLE IF NOT EXISTS syncpoint (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `primary_ts` ) )") // todo err 处理 } @@ -914,7 +915,7 @@ func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { tx.Rollback() return err } - tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, slaveTs) + tx.Exec("insert into TiCDC.syncpoint(cf, primary_ts, secondary_ts) VALUES (?,?,?)", c.id, c.status.CheckpointTs, slaveTs) //tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, 0) tx.Commit() //TODO 处理错误 return nil diff --git a/cdc/owner.go b/cdc/owner.go index 078654691b9..90b2b8f990a 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -15,17 +15,14 @@ package cdc import ( "context" - "database/sql" "fmt" "io" "math" - "net/url" "os" "strings" "sync" "time" - dmysql "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" @@ -309,7 +306,7 @@ func (o *Owner) newChangeFeed( } errCh := make(chan error, 1) - sink, err := sink.NewSink(ctx, id, info.SinkURI, filter, info.Config, info.Opts, errCh) + primarySink, err := sink.NewSink(ctx, id, info.SinkURI, filter, info.Config, info.Opts, errCh) if err != nil { cancel() return nil, errors.Trace(err) @@ -324,12 +321,12 @@ func (o *Owner) newChangeFeed( cancel() }() - err = sink.Initialize(ctx, sinkTableInfo) + err = primarySink.Initialize(ctx, sinkTableInfo) if err != nil { log.Error("error on running owner", zap.Error(err)) } - syncDB, err := o.createSyncpointSink(ctx, info, id) + syncDB, err := sink.NewSyncpointSinklink(ctx, info, id) if err != nil { log.Error("error on running owner", zap.Error(err)) } @@ -360,7 +357,7 @@ func (o *Owner) newChangeFeed( taskPositions: taskPositions, etcdCli: o.etcdClient, filter: filter, - sink: sink, + sink: primarySink, cyclicEnabled: info.Config.Cyclic.IsEnabled(), lastRebalanceTime: time.Now(), } @@ -509,7 +506,7 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { //create the sync table newCf.createSynctable() //start SyncPeriod for create the sync point - newCf.startSyncPeriod() + newCf.startSyncPeriod(ctx) delete(o.stoppedFeeds, changeFeedID) } o.adminJobsLock.Lock() @@ -1268,116 +1265,47 @@ func (o *Owner) startCaptureWatcher(ctx context.Context) { }() } -func (o *Owner) configureSinkURI(ctx context.Context, dsnCfg *dmysql.Config, tz *time.Location) (string, error) { - if dsnCfg.Params == nil { - dsnCfg.Params = make(map[string]string, 1) - } - dsnCfg.DBName = "" - dsnCfg.InterpolateParams = true - dsnCfg.MultiStatements = true - dsnCfg.Params["time_zone"] = fmt.Sprintf(`"%s"`, tz.String()) - - testDB, err := sql.Open("mysql", dsnCfg.FormatDSN()) - if err != nil { - return "", errors.Annotate(err, "fail to open MySQL connection when configuring sink") - } - defer testDB.Close() - log.Debug("Opened connection to configure some tidb special parameters") - - var variableName string - var autoRandomInsertEnabled string - queryStr := "show session variables like 'allow_auto_random_explicit_insert';" - err = testDB.QueryRowContext(ctx, queryStr).Scan(&variableName, &autoRandomInsertEnabled) - if err != nil && err != sql.ErrNoRows { - return "", errors.Annotate(err, "fail to query sink for support of auto-random") - } - if err == nil && (autoRandomInsertEnabled == "off" || autoRandomInsertEnabled == "0") { - dsnCfg.Params["allow_auto_random_explicit_insert"] = "1" - log.Debug("Set allow_auto_random_explicit_insert to 1") - } +/*func (o *Owner) configureSinkURI(ctx context.Context, dsnCfg *dmysql.Config, tz *time.Location) (string, error) { +if dsnCfg.Params == nil { + dsnCfg.Params = make(map[string]string, 1) +} +dsnCfg.DBName = "" +dsnCfg.InterpolateParams = true +dsnCfg.MultiStatements = true +dsnCfg.Params["time_zone"] = fmt.Sprintf(`"%s"`, tz.String()) + +testDB, err := sql.Open("mysql", dsnCfg.FormatDSN()) +if err != nil { + return "", errors.Annotate(err, "fail to open MySQL connection when configuring sink") +} +defer testDB.Close() +log.Debug("Opened connection to configure some tidb special parameters") + +var variableName string +var autoRandomInsertEnabled string +queryStr := "show session variables like 'allow_auto_random_explicit_insert';" +err = testDB.QueryRowContext(ctx, queryStr).Scan(&variableName, &autoRandomInsertEnabled) +if err != nil && err != sql.ErrNoRows { + return "", errors.Annotate(err, "fail to query sink for support of auto-random") +} +if err == nil && (autoRandomInsertEnabled == "off" || autoRandomInsertEnabled == "0") { + dsnCfg.Params["allow_auto_random_explicit_insert"] = "1" + log.Debug("Set allow_auto_random_explicit_insert to 1") +} - var txnMode string - queryStr = "show session variables like 'tidb_txn_mode';" - err = testDB.QueryRowContext(ctx, queryStr).Scan(&variableName, &txnMode) - if err != nil && err != sql.ErrNoRows { - return "", errors.Annotate(err, "fail to query sink for txn mode") - } - /*if err == nil { - dsnCfg.Params["tidb_txn_mode"] = params.tidbTxnMode - }*/ +var txnMode string +queryStr = "show session variables like 'tidb_txn_mode';" +err = testDB.QueryRowContext(ctx, queryStr).Scan(&variableName, &txnMode) +if err != nil && err != sql.ErrNoRows { + return "", errors.Annotate(err, "fail to query sink for txn mode") +} +/*if err == nil { + dsnCfg.Params["tidb_txn_mode"] = params.tidbTxnMode +}*/ - dsnClone := dsnCfg.Clone() +/* dsnClone := dsnCfg.Clone() dsnClone.Passwd = "******" log.Info("sink uri is configured", zap.String("format dsn", dsnClone.FormatDSN())) return dsnCfg.FormatDSN(), nil -} - -//createSyncpointSink create a sink to record the syncpoint map in downstream DB for every changefeed -func (o *Owner) createSyncpointSink(ctx context.Context, info *model.ChangeFeedInfo, id string) (*sql.DB, error) { - var syncDB *sql.DB - // parse sinkURI as a URI - sinkURI, err := url.Parse(info.SinkURI) - if err != nil { - return nil, errors.Annotatef(err, "parse sinkURI failed") - } - if sinkURI == nil { - return nil, errors.New("fail to open MySQL sink, empty URL") - } - scheme := strings.ToLower(sinkURI.Scheme) - if scheme != "mysql" && scheme != "tidb" { - return nil, errors.New("can create mysql sink with unsupported scheme") - } - var tlsParam string - if sinkURI.Query().Get("ssl-ca") != "" { - credential := security.Credential{ - CAPath: sinkURI.Query().Get("ssl-ca"), - CertPath: sinkURI.Query().Get("ssl-cert"), - KeyPath: sinkURI.Query().Get("ssl-key"), - } - tlsCfg, err := credential.ToTLSConfig() - if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") - } - name := "cdc_mysql_tls" + "syncpoint" + id - err = dmysql.RegisterTLSConfig(name, tlsCfg) - if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") - } - tlsParam = "?tls=" + name - } - - // dsn format of the driver: - // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] - username := sinkURI.User.Username() - password, _ := sinkURI.User.Password() - port := sinkURI.Port() - if username == "" { - username = "root" - } - if port == "" { - port = "4000" - } - - dsnStr := fmt.Sprintf("%s:%s@tcp(%s:%s)/%s", username, password, sinkURI.Hostname(), port, tlsParam) - dsn, err := dmysql.ParseDSN(dsnStr) - if err != nil { - return nil, errors.Trace(err) - } - dsnStr, err = o.configureSinkURI(ctx, dsn, util.TimezoneFromCtx(ctx)) - if err != nil { - return nil, errors.Trace(err) - } - syncDB, err = sql.Open("mysql", dsnStr) - if err != nil { - return nil, errors.Annotate(err, "Open database connection failed") - } - err = syncDB.PingContext(ctx) - if err != nil { - return nil, errors.Annotatef(err, "fail to open MySQL connection") - } - - log.Info("Start mysql syncpoint sink") - - return syncDB, nil -} +}*/ diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index d7b226d6dfd..49796106977 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -842,3 +842,79 @@ func buildColumnList(names []string) string { return b.String() } + +//NewSyncpointSinklink create a sink to record the syncpoint map in downstream DB for every changefeed +func NewSyncpointSinklink(ctx context.Context, info *model.ChangeFeedInfo, id string) (*sql.DB, error) { + var syncDB *sql.DB + // parse sinkURI as a URI + sinkURI, err := url.Parse(info.SinkURI) + if err != nil { + return nil, errors.Annotatef(err, "parse sinkURI failed") + } + //todo If is neither mysql nor tidb, such as kafka, just ignore this feature. + scheme := strings.ToLower(sinkURI.Scheme) + if scheme != "mysql" && scheme != "tidb" { + return nil, errors.New("can create mysql sink with unsupported scheme") + } + params := defaultParams + s := sinkURI.Query().Get("tidb-txn-mode") + if s != "" { + if s == "pessimistic" || s == "optimistic" { + params.tidbTxnMode = s + } else { + log.Warn("invalid tidb-txn-mode, should be pessimistic or optimistic, use optimistic as default") + } + } + var tlsParam string + if sinkURI.Query().Get("ssl-ca") != "" { + credential := security.Credential{ + CAPath: sinkURI.Query().Get("ssl-ca"), + CertPath: sinkURI.Query().Get("ssl-cert"), + KeyPath: sinkURI.Query().Get("ssl-key"), + } + tlsCfg, err := credential.ToTLSConfig() + if err != nil { + return nil, errors.Annotate(err, "fail to open MySQL connection") + } + name := "cdc_mysql_tls" + "syncpoint" + id + err = dmysql.RegisterTLSConfig(name, tlsCfg) + if err != nil { + return nil, errors.Annotate(err, "fail to open MySQL connection") + } + tlsParam = "?tls=" + name + } + + // dsn format of the driver: + // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] + username := sinkURI.User.Username() + password, _ := sinkURI.User.Password() + port := sinkURI.Port() + if username == "" { + username = "root" + } + if port == "" { + port = "4000" + } + + dsnStr := fmt.Sprintf("%s:%s@tcp(%s:%s)/%s", username, password, sinkURI.Hostname(), port, tlsParam) + dsn, err := dmysql.ParseDSN(dsnStr) + if err != nil { + return nil, errors.Trace(err) + } + dsnStr, err = configureSinkURI(ctx, dsn, util.TimezoneFromCtx(ctx), params) + if err != nil { + return nil, errors.Trace(err) + } + syncDB, err = sql.Open("mysql", dsnStr) + if err != nil { + return nil, errors.Annotate(err, "Open database connection failed") + } + err = syncDB.PingContext(ctx) + if err != nil { + return nil, errors.Annotatef(err, "fail to open MySQL connection") + } + + log.Info("Start mysql syncpoint sink") + + return syncDB, nil +} From abd124bfaf2b7d9ad286b826d5b42aa0043169d0 Mon Sep 17 00:00:00 2001 From: colin Date: Sat, 12 Sep 2020 00:45:08 +0800 Subject: [PATCH 10/35] add integration test for sync point record --- cdc/changefeed.go | 11 +- scripts/fix_lib_zstd.sh | 1 + tests/syncpoint/conf/diff_config_final.toml | 27 ++ tests/syncpoint/conf/diff_config_part1.toml | 23 ++ tests/syncpoint/conf/diff_config_part2.toml | 8 + tests/syncpoint/conf/workload | 13 + tests/syncpoint/deploy_config.go | 56 +++ tests/syncpoint/run.sh | 121 +++++++ tests/syncpoint/test.sh | 8 + tests/util/diff_config.go | 371 ++++++++++++++++++++ 10 files changed, 636 insertions(+), 3 deletions(-) create mode 100644 tests/syncpoint/conf/diff_config_final.toml create mode 100644 tests/syncpoint/conf/diff_config_part1.toml create mode 100644 tests/syncpoint/conf/diff_config_part2.toml create mode 100644 tests/syncpoint/conf/workload create mode 100644 tests/syncpoint/deploy_config.go create mode 100755 tests/syncpoint/run.sh create mode 100755 tests/syncpoint/test.sh create mode 100644 tests/util/diff_config.go diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 4c60b1d57d3..ba38ad8b4f5 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -35,7 +35,8 @@ import ( "go.uber.org/zap" ) -const syncInterval = time.Minute * 10 //Interval for set syncpoint, todo: parameterization +//const syncInterval = time.Minute * 10 //Interval for set syncpoint, todo: parameterization +const syncInterval = time.Second * 10 //Interval for set syncpoint, todo: parameterization type tableIDMap = map[model.TableID]struct{} @@ -719,7 +720,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { } //if (c.status.ResolvedTs == c.status.CheckpointTs && (c.updateResolvedTs == false || c.status.ResolvedTs == c.ddlResolvedTs)) || c.status.ResolvedTs == 0 { - if (c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs) || c.status.ResolvedTs == 0 { + if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { log.Debug("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs)) //c.syncPointTs = c.targetTs //恢复syncPointTs,使得ResoledTs可以继续推进 log.Info("achive the sync point with timer", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) @@ -727,6 +728,10 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { c.sinkSyncpoint(ctx) } + if c.status.ResolvedTs == 0 { + c.updateResolvedTs = true + } + if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs { log.Info("achive the sync point with ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) c.sinkSyncpoint(ctx) @@ -917,6 +922,6 @@ func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { } tx.Exec("insert into TiCDC.syncpoint(cf, primary_ts, secondary_ts) VALUES (?,?,?)", c.id, c.status.CheckpointTs, slaveTs) //tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, 0) - tx.Commit() //TODO 处理错误 + tx.Commit() //TODO deal with error return nil } diff --git a/scripts/fix_lib_zstd.sh b/scripts/fix_lib_zstd.sh index 9df1fb2e4a4..eec0d60bce0 100755 --- a/scripts/fix_lib_zstd.sh +++ b/scripts/fix_lib_zstd.sh @@ -3,6 +3,7 @@ user=$(whoami) GOOS=$(go env GOOS) GOARCH=$(go env GOARCH) +GOPATH=$(go env GOPATH) module="github.com/valyala/gozstd@v1.7.0" GO111MODULE=on go mod download ${module} diff --git a/tests/syncpoint/conf/diff_config_final.toml b/tests/syncpoint/conf/diff_config_final.toml new file mode 100644 index 00000000000..88b51f6546c --- /dev/null +++ b/tests/syncpoint/conf/diff_config_final.toml @@ -0,0 +1,27 @@ +# diff Configuration. + +log-level = "info" +chunk-size = 10 +check-thread-count = 4 +sample-percent = 100 +use-checksum = true +fix-sql-file = "fix.sql" + +# tables need to check. +[[check-tables]] + schema = "testSync" + tables = ["usertable","simple1","simple2"] + +[[source-db]] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + instance-id = "source-1" + +[target-db] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" + instance-id = "target-1" \ No newline at end of file diff --git a/tests/syncpoint/conf/diff_config_part1.toml b/tests/syncpoint/conf/diff_config_part1.toml new file mode 100644 index 00000000000..9ceb944eb58 --- /dev/null +++ b/tests/syncpoint/conf/diff_config_part1.toml @@ -0,0 +1,23 @@ +# diff Configuration. + +log-level = "info" +chunk-size = 10 +check-thread-count = 4 +sample-percent = 100 +use-checksum = true +fix-sql-file = "fix.sql" + +# tables need to check. +[[check-tables]] + schema = "testSync" + #tables = ["usertable","simple1","simple2"] + #tables = ["usertable"] + tables = ["~^"] + +[[source-db]] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + instance-id = "source-1" + \ No newline at end of file diff --git a/tests/syncpoint/conf/diff_config_part2.toml b/tests/syncpoint/conf/diff_config_part2.toml new file mode 100644 index 00000000000..e3c8d8dd60e --- /dev/null +++ b/tests/syncpoint/conf/diff_config_part2.toml @@ -0,0 +1,8 @@ + +[target-db] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" + instance-id = "target-1" + \ No newline at end of file diff --git a/tests/syncpoint/conf/workload b/tests/syncpoint/conf/workload new file mode 100644 index 00000000000..7649fd9afb7 --- /dev/null +++ b/tests/syncpoint/conf/workload @@ -0,0 +1,13 @@ +threadcount=2 +recordcount=10 +operationcount=0 +workload=core + +readallfields=true + +readproportion=0 +updateproportion=0 +scanproportion=0 +insertproportion=0 + +requestdistribution=uniform diff --git a/tests/syncpoint/deploy_config.go b/tests/syncpoint/deploy_config.go new file mode 100644 index 00000000000..0717781c72b --- /dev/null +++ b/tests/syncpoint/deploy_config.go @@ -0,0 +1,56 @@ +// Copyright 2020 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 main + +import ( + "bytes" + "flag" + "fmt" + "os" + + "github.com/BurntSushi/toml" + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/tests/util" +) + +func main() { + if len(os.Args) != 5 { + log.Info("wrong args,need three args!") + os.Exit(2) + } + primaryTs := os.Args[1] + secondaryTs := os.Args[2] + diffConfig := util.NewDiffConfig() + err := diffConfig.Parse(os.Args[3:]) + switch errors.Cause(err) { + case nil: + case flag.ErrHelp: + os.Exit(0) + default: + log.S().Errorf("parse cmd flags err %s\n", err) + os.Exit(2) + } + // if len(os.Args) != 3 { + // log.S().Errorf("wrong args,need two args %s\n", errors.New("wrong args")) + // os.Exit(2) + // } + diffConfig.SourceDBCfg[0].Snapshot = primaryTs + diffConfig.TargetDBCfg.Snapshot = secondaryTs + buf := new(bytes.Buffer) + if err := toml.NewEncoder(buf).Encode(diffConfig); err != nil { + log.Fatal("someting wrong") + } + fmt.Println(buf.String()) +} diff --git a/tests/syncpoint/run.sh b/tests/syncpoint/run.sh new file mode 100755 index 00000000000..72a37c44803 --- /dev/null +++ b/tests/syncpoint/run.sh @@ -0,0 +1,121 @@ +#!/bin/bash + +set -e + +CUR=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +source $CUR/../_utils/test_prepare +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +CDC_COUNT=3 +DB_COUNT=4 + +function ddl() { + run_sql "DROP table IF EXISTS testSync.simple1" + sleep 2 + run_sql "DROP table IF EXISTS testSync.simple2" + sleep 2 + run_sql "CREATE table testSync.simple1(id int primary key, val int);" + sleep 2 + run_sql "INSERT INTO testSync.simple1(id, val) VALUES (1, 1);" + sleep 2 + run_sql "INSERT INTO testSync.simple1(id, val) VALUES (2, 2);" + sleep 2 + run_sql "INSERT INTO testSync.simple1(id, val) VALUES (3, 3);" + sleep 2 + run_sql "CREATE table testSync.simple2(id int primary key, val int);" + sleep 2 + run_sql "INSERT INTO testSync.simple2(id, val) VALUES (1, 1);" + sleep 2 + run_sql "INSERT INTO testSync.simple2(id, val) VALUES (2, 2);" + sleep 2 + run_sql "INSERT INTO testSync.simple2(id, val) VALUES (3, 3);" + sleep 2 + run_sql "CREATE index simple1_val ON testSync.simple1(val);" + sleep 2 + run_sql "CREATE index simple2_val ON testSync.simple2(val);" + sleep 2 + run_sql "DELETE FROM testSync.simple1 where id=1;" + sleep 2 + run_sql "DELETE FROM testSync.simple2 where id=1;" + sleep 2 + run_sql "DELETE FROM testSync.simple1 where id=2;" + sleep 2 + run_sql "DELETE FROM testSync.simple2 where id=2;" + sleep 2 + run_sql "DROP index simple1_val ON testSync.simple1;" + sleep 2 + run_sql "DROP index simple2_val ON testSync.simple2;" + sleep 2 +} + +function goSql() { + for i in {1..3} + do + go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=testSync + sleep 2 + ddl + sleep 2 + done +} + +function deployConfig() { + cat $CUR/conf/diff_config_part1.toml > $CUR/conf/diff_config.toml + echo "snapshot = \"$1\"" >> $CUR/conf/diff_config.toml + cat $CUR/conf/diff_config_part2.toml >> $CUR/conf/diff_config.toml + echo "snapshot = \"$2\"" >> $CUR/conf/diff_config.toml +} + +function checkDiff() { + primaryArr=(`grep primary_ts $OUT_DIR/sql_res.$TEST_NAME.txt | awk -F ": " '{print $2}'`) + secondaryArr=(`grep secondary_ts $OUT_DIR/sql_res.$TEST_NAME.txt | awk -F ": " '{print $2}'`) + num=${#primaryArr[*]} + for ((i=0;i<$num;i++)) + do + deployConfig ${primaryArr[$i]} ${secondaryArr[$i]} + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml + done + rm $CUR/conf/diff_config.toml +} + +function run() { + if [ "$SINK_TYPE" != "mysql" ]; then + echo "kafka downstream isn't support syncpoint record" + return + fi + + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + run_sql "CREATE DATABASE testSync;" + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" + + goSql + + check_table_exists "testSync.USERTABLE" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "testSync.simple1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "testSync.simple2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + sleep 60 + + run_sql "SELECT primary_ts, secondary_ts FROM TiCDC.syncpoint;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + echo "____________________________________" + cat "$OUT_DIR/sql_res.$TEST_NAME.txt" + checkDiff + check_sync_diff $WORK_DIR $CUR/conf/diff_config_final.toml + + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +run $* +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/syncpoint/test.sh b/tests/syncpoint/test.sh new file mode 100755 index 00000000000..8c591fcdf3a --- /dev/null +++ b/tests/syncpoint/test.sh @@ -0,0 +1,8 @@ + +function deployConfig() { + cat ./conf/diff_config_part1.toml > ./conf/diff_config.toml + echo "snapshot = "$1"" >> ./conf/diff_config.toml + cat ./conf/diff_config_part2.toml >> ./conf/diff_config.toml + echo "snapshot = "$2"" >> ./conf/diff_config.toml +} +deployConfig 123 456 diff --git a/tests/util/diff_config.go b/tests/util/diff_config.go new file mode 100644 index 00000000000..17ec90eeb30 --- /dev/null +++ b/tests/util/diff_config.go @@ -0,0 +1,371 @@ +// Copyright 2020 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 util + +import ( + "database/sql" + "encoding/json" + "flag" + "net/url" + "strconv" + + "github.com/BurntSushi/toml" + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb-tools/pkg/dbutil" + router "github.com/pingcap/tidb-tools/pkg/table-router" + "go.uber.org/zap" +) + +const ( + percent0 = 0 + percent100 = 100 +) + +var sourceInstanceMap map[string]interface{} = make(map[string]interface{}) + +// DBConfig is the config of database, and keep the connection. +type DiffDBConfig struct { + dbutil.DBConfig + + InstanceID string `toml:"instance-id" json:"instance-id"` + + Conn *sql.DB +} + +// Valid returns true if database's config is valide. +func (c *DiffDBConfig) Valid() bool { + if c.InstanceID == "" { + log.Error("must specify source database's instance id") + return false + } + sourceInstanceMap[c.InstanceID] = struct{}{} + + return true +} + +// CheckTables saves the tables need to check. +type DiffCheckTables struct { + // schema name + Schema string `toml:"schema" json:"schema"` + + // table list + Tables []string `toml:"tables" json:"tables"` + + ExcludeTables []string `toml:"exclude-tables" json:"exclude-tables"` +} + +// TableConfig is the config of table. +type DiffTableConfig struct { + // table's origin information + DiffTableInstance + // columns be ignored, will not check this column's data + IgnoreColumns []string `toml:"ignore-columns"` + // field should be the primary key, unique key or field with index + Fields string `toml:"index-fields"` + // select range, for example: "age > 10 AND age < 20" + Range string `toml:"range"` + // set true if comparing sharding tables with target table, should have more than one source tables. + IsSharding bool `toml:"is-sharding"` + // saves the source tables's info. + // may have more than one source for sharding tables. + // or you want to compare table with different schema and table name. + // SourceTables can be nil when source and target is one-to-one correspondence. + SourceTables []DiffTableInstance `toml:"source-tables"` + TargetTableInfo *model.TableInfo + + // collation config in mysql/tidb + Collation string `toml:"collation"` +} + +// Valid returns true if table's config is valide. +func (t *DiffTableConfig) Valid() bool { + if t.Schema == "" || t.Table == "" { + log.Error("schema and table's name can't be empty") + return false + } + + if t.IsSharding { + if len(t.SourceTables) <= 1 { + log.Error("must have more than one source tables if comparing sharding tables") + return false + } + + } else { + if len(t.SourceTables) > 1 { + log.Error("have more than one source table in no sharding mode") + return false + } + } + + for _, sourceTable := range t.SourceTables { + if !sourceTable.Valid() { + return false + } + } + + return true +} + +// TableInstance saves the base information of table. +type DiffTableInstance struct { + // database's instance id + InstanceID string `toml:"instance-id" json:"instance-id"` + // schema name + Schema string `toml:"schema"` + // table name + Table string `toml:"table"` +} + +// Valid returns true if table instance's info is valide. +// should be executed after source database's check. +func (t *DiffTableInstance) Valid() bool { + if t.InstanceID == "" { + log.Error("must specify the database's instance id for source table") + return false + } + + if _, ok := sourceInstanceMap[t.InstanceID]; !ok { + log.Error("unknown database instance id", zap.String("instance id", t.InstanceID)) + return false + } + + if t.Schema == "" || t.Table == "" { + log.Error("schema and table's name can't be empty") + return false + } + + return true +} + +// Config is the configuration. +type DiffConfig struct { + *flag.FlagSet `json:"-"` + + // log level + LogLevel string `toml:"log-level" json:"log-level"` + + // source database's config + SourceDBCfg []DiffDBConfig `toml:"source-db" json:"source-db"` + + // target database's config + TargetDBCfg DiffDBConfig `toml:"target-db" json:"target-db"` + + // for example, the whole data is [1...100] + // we can split these data to [1...10], [11...20], ..., [91...100] + // the [1...10] is a chunk, and it's chunk size is 10 + // size of the split chunk + ChunkSize int `toml:"chunk-size" json:"chunk-size"` + + // sampling check percent, for example 10 means only check 10% data + Sample int `toml:"sample-percent" json:"sample-percent"` + + // how many goroutines are created to check data + CheckThreadCount int `toml:"check-thread-count" json:"check-thread-count"` + + // set false if want to comapre the data directly + UseChecksum bool `toml:"use-checksum" json:"use-checksum"` + + // set true if just want compare data by checksum, will skip select data when checksum is not equal. + OnlyUseChecksum bool `toml:"only-use-checksum" json:"only-use-checksum"` + + // the name of the file which saves sqls used to fix different data + FixSQLFile string `toml:"fix-sql-file" json:"fix-sql-file"` + + // the tables to be checked + Tables []*DiffCheckTables `toml:"check-tables" json:"check-tables"` + + // TableRules defines table name and database name's conversion relationship between source database and target database + TableRules []*router.TableRule `toml:"table-rules" json:"table-rules"` + + // the config of table + TableCfgs []*DiffTableConfig `toml:"table-config" json:"table-config"` + + // ignore check table's struct + IgnoreStructCheck bool `toml:"ignore-struct-check" json:"ignore-struct-check"` + + // ignore check table's data + IgnoreDataCheck bool `toml:"ignore-data-check" json:"ignore-data-check"` + + // set true will continue check from the latest checkpoint + UseCheckpoint bool `toml:"use-checkpoint" json:"use-checkpoint"` + + // DMAddr is dm-master's address, the format should like "http://127.0.0.1:8261" + DMAddr string `toml:"dm-addr" json:"dm-addr"` + // DMTask is dm's task name + DMTask string `toml:"dm-task" json:"dm-task"` + + // config file + ConfigFile string + + // print version if set true + PrintVersion bool +} + +// NewConfig creates a new config. +func NewDiffConfig() *DiffConfig { + cfg := &DiffConfig{} + cfg.FlagSet = flag.NewFlagSet("diff", flag.ContinueOnError) + fs := cfg.FlagSet + + fs.StringVar(&cfg.ConfigFile, "config", "", "Config file") + fs.StringVar(&cfg.LogLevel, "L", "info", "log level: debug, info, warn, error, fatal") + fs.IntVar(&cfg.ChunkSize, "chunk-size", 1000, "diff check chunk size") + fs.IntVar(&cfg.Sample, "sample", 100, "the percent of sampling check") + fs.IntVar(&cfg.CheckThreadCount, "check-thread-count", 1, "how many goroutines are created to check data") + fs.BoolVar(&cfg.UseChecksum, "use-checksum", true, "set false if want to comapre the data directly") + fs.StringVar(&cfg.FixSQLFile, "fix-sql-file", "fix.sql", "the name of the file which saves sqls used to fix different data") + fs.BoolVar(&cfg.PrintVersion, "V", false, "print version of sync_diff_inspector") + fs.BoolVar(&cfg.IgnoreDataCheck, "ignore-data-check", false, "ignore check table's data") + fs.BoolVar(&cfg.IgnoreStructCheck, "ignore-struct-check", false, "ignore check table's struct") + fs.BoolVar(&cfg.UseCheckpoint, "use-checkpoint", true, "set true will continue check from the latest checkpoint") + + return cfg +} + +// Parse parses flag definitions from the argument list. +func (c *DiffConfig) Parse(arguments []string) error { + // Parse first to get config file. + err := c.FlagSet.Parse(arguments) + if err != nil { + return errors.Trace(err) + } + + // Load config file if specified. + if c.ConfigFile != "" { + err = c.configFromFile(c.ConfigFile) + if err != nil { + return errors.Trace(err) + } + } + + // Parse again to replace with command line options. + err = c.FlagSet.Parse(arguments) + if err != nil { + return errors.Trace(err) + } + + if len(c.FlagSet.Args()) != 0 { + return errors.Errorf("'%s' is an invalid flag", c.FlagSet.Arg(0)) + } + + return nil +} + +func (c *DiffConfig) String() string { + cfg, err := json.Marshal(c) + if err != nil { + return "" + } + return string(cfg) +} + +// configFromFile loads config from file. +func (c *DiffConfig) configFromFile(path string) error { + _, err := toml.DecodeFile(path, c) + return errors.Trace(err) +} + +func (c *DiffConfig) configToFile(path string) error { + _, err := toml.DecodeFile(path, c) + return errors.Trace(err) +} + +func (c *DiffConfig) checkConfig() bool { + if c.Sample > percent100 || c.Sample < percent0 { + log.Error("sample must be greater than 0 and less than or equal to 100!") + return false + } + + if c.CheckThreadCount <= 0 { + log.Error("check-thcount must greater than 0!") + return false + } + + if len(c.DMAddr) != 0 { + u, err := url.Parse(c.DMAddr) + if err != nil || u.Scheme == "" || u.Host == "" { + log.Error("dm-addr's format should like 'http://127.0.0.1:8261'") + return false + } + + if len(c.DMTask) == 0 { + log.Error("must set the `dm-task` if set `dm-addr`") + return false + } + + emptyDBConfig := DiffDBConfig{} + // source DB, target DB and check table's information will get from DM, should not set them + if len(c.SourceDBCfg) != 0 || c.TargetDBCfg != emptyDBConfig { + log.Error("should not set `source-db` or `target-db`, diff will generate them automatically when set `dm-addr` and `dm-task`") + return false + } + + if len(c.Tables) != 0 || len(c.TableRules) != 0 || len(c.TableCfgs) != 0 { + log.Error("should not set `check-tables`, `table-rules` or `table-config`, diff will generate them automatically when set `dm-addr` and `dm-task`") + return false + } + } else { + if len(c.SourceDBCfg) == 0 { + log.Error("must have at least one source database") + return false + } + + for i := range c.SourceDBCfg { + if !c.SourceDBCfg[i].Valid() { + return false + } + if c.SourceDBCfg[i].Snapshot != "" { + c.SourceDBCfg[i].Snapshot = strconv.Quote(c.SourceDBCfg[i].Snapshot) + } + } + + if c.TargetDBCfg.InstanceID == "" { + c.TargetDBCfg.InstanceID = "target" + } + if c.TargetDBCfg.Snapshot != "" { + c.TargetDBCfg.Snapshot = strconv.Quote(c.TargetDBCfg.Snapshot) + } + if _, ok := sourceInstanceMap[c.TargetDBCfg.InstanceID]; ok { + log.Error("target has same instance id in source", zap.String("instance id", c.TargetDBCfg.InstanceID)) + return false + } + + if len(c.Tables) == 0 { + log.Error("must specify check tables") + return false + } + + for _, tableCfg := range c.TableCfgs { + if !tableCfg.Valid() { + return false + } + } + } + + if c.OnlyUseChecksum { + if !c.UseChecksum { + log.Error("need set use-checksum = true") + return false + } + } else { + if len(c.FixSQLFile) == 0 { + log.Warn("fix-sql-file is invalid, will use default value 'fix.sql'") + c.FixSQLFile = "fix.sql" + } + } + + return true +} From 091927bf84822fde2b097cf42d96edbf9dffa8d2 Mon Sep 17 00:00:00 2001 From: colin Date: Sun, 13 Sep 2020 18:52:52 +0800 Subject: [PATCH 11/35] 9.13 add sync-point and sync-interval flag for 'cdc cli changefeed create' --- cdc/changefeed.go | 61 ++--- cdc/model/changefeed.go | 3 + cdc/owner.go | 18 +- cmd/client.go | 3 + cmd/client_changefeed.go | 22 +- tests/syncpoint/deploy_config.go | 56 ----- tests/syncpoint/run.sh | 2 +- tests/syncpoint/test.sh | 8 - tests/util/diff_config.go | 371 ------------------------------- 9 files changed, 69 insertions(+), 475 deletions(-) delete mode 100644 tests/syncpoint/deploy_config.go delete mode 100755 tests/syncpoint/test.sh delete mode 100644 tests/util/diff_config.go diff --git a/cdc/changefeed.go b/cdc/changefeed.go index ba38ad8b4f5..f6a276b9890 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -35,8 +35,7 @@ import ( "go.uber.org/zap" ) -//const syncInterval = time.Minute * 10 //Interval for set syncpoint, todo: parameterization -const syncInterval = time.Second * 10 //Interval for set syncpoint, todo: parameterization +const defaultSyncInterval = time.Minute * 10 //default Interval for record syncpoint type tableIDMap = map[model.TableID]struct{} @@ -719,23 +718,26 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { } } - //if (c.status.ResolvedTs == c.status.CheckpointTs && (c.updateResolvedTs == false || c.status.ResolvedTs == c.ddlResolvedTs)) || c.status.ResolvedTs == 0 { - if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { - log.Debug("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs)) - //c.syncPointTs = c.targetTs //恢复syncPointTs,使得ResoledTs可以继续推进 - log.Info("achive the sync point with timer", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) - c.updateResolvedTs = true - c.sinkSyncpoint(ctx) - } + //sync-point on + if c.info.SyncPoint { + //if (c.status.ResolvedTs == c.status.CheckpointTs && (c.updateResolvedTs == false || c.status.ResolvedTs == c.ddlResolvedTs)) || c.status.ResolvedTs == 0 { + if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { + log.Debug("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs)) + //c.syncPointTs = c.targetTs //恢复syncPointTs,使得ResoledTs可以继续推进 + log.Info("achive the sync point with timer", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) + c.updateResolvedTs = true + c.sinkSyncpoint(ctx) + } - if c.status.ResolvedTs == 0 { - c.updateResolvedTs = true - } + if c.status.ResolvedTs == 0 { + c.updateResolvedTs = true + } - if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs { - log.Info("achive the sync point with ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) - c.sinkSyncpoint(ctx) - c.ddlTs = 0 + if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs { + log.Info("achive the sync point with ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) + c.sinkSyncpoint(ctx) + c.ddlTs = 0 + } } if len(c.taskPositions) < len(c.taskStatus) { @@ -831,15 +833,21 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { var tsUpdated bool - if c.updateResolvedTs && minResolvedTs > c.status.ResolvedTs { - //prevResolvedTs := c.status.ResolvedTs + //syncpoint on + if c.info.SyncPoint { + if c.updateResolvedTs && minResolvedTs > c.status.ResolvedTs { + //prevResolvedTs := c.status.ResolvedTs + c.status.ResolvedTs = minResolvedTs + tsUpdated = true + /*if prevResolvedTs == c.status.CheckpointTs || prevResolvedTs == 0 { + //到达sync point + //todo 需要重新开始启动计时 + c.startTimer <- true + }*/ + } + } else if minResolvedTs > c.status.ResolvedTs { c.status.ResolvedTs = minResolvedTs tsUpdated = true - /*if prevResolvedTs == c.status.CheckpointTs || prevResolvedTs == 0 { - //到达sync point - //todo 需要重新开始启动计时 - c.startTimer <- true - }*/ } if minCheckpointTs > c.status.CheckpointTs { @@ -882,9 +890,10 @@ func (c *changeFeed) pullDDLJob() error { } // startSyncPeriod start a timer for every changefeed to create sync point by time -func (c *changeFeed) startSyncPeriod(ctx context.Context) { +func (c *changeFeed) startSyncPeriod(ctx context.Context, interval time.Duration) { + log.Debug("sync ticker start", zap.Duration("sync-interval", interval)) go func(ctx context.Context) { - ticker := time.NewTicker(syncInterval) + ticker := time.NewTicker(interval) for { select { case <-ctx.Done(): diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index d04692a07ee..f5ded07485f 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -78,6 +78,9 @@ type ChangeFeedInfo struct { State FeedState `json:"state"` ErrorHis []int64 `json:"history"` Error *RunningError `json:"error"` + + SyncPoint bool `json:"sync-point"` + SyncInterval string `json:"sync-interval"` } var changeFeedIDRe *regexp.Regexp = regexp.MustCompile(`^[a-zA-Z0-9]+(\-[a-zA-Z0-9]+)*$`) diff --git a/cdc/owner.go b/cdc/owner.go index 90b2b8f990a..a795b6a2ce3 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -503,10 +503,20 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { continue } o.changeFeeds[changeFeedID] = newCf - //create the sync table - newCf.createSynctable() - //start SyncPeriod for create the sync point - newCf.startSyncPeriod(ctx) + if newCf.info.SyncPoint { + log.Info("syncpoint is on", zap.Bool("syncpoint", newCf.info.SyncPoint)) + //create the sync table + newCf.createSynctable() + //start SyncPeriod for create the sync point + syncInterval, err := time.ParseDuration(newCf.info.SyncInterval) + if err != nil { + syncInterval = defaultSyncInterval + } + newCf.startSyncPeriod(ctx, syncInterval) + } else { + log.Info("syncpoint is off", zap.Bool("syncpoint", newCf.info.SyncPoint)) + } + delete(o.stoppedFeeds, changeFeedID) } o.adminJobsLock.Lock() diff --git a/cmd/client.go b/cmd/client.go index 4d936e7ee5e..1944488b3ca 100644 --- a/cmd/client.go +++ b/cmd/client.go @@ -72,6 +72,9 @@ var ( captureID string interval uint + syncPoint bool + syncInterval string + defaultContext context.Context ) diff --git a/cmd/client_changefeed.go b/cmd/client_changefeed.go index 9d025a8922f..78c5466d0ab 100644 --- a/cmd/client_changefeed.go +++ b/cmd/client_changefeed.go @@ -230,15 +230,17 @@ func verifyChangefeedParamers(ctx context.Context, cmd *cobra.Command, isCreate } } info := &model.ChangeFeedInfo{ - SinkURI: sinkURI, - Opts: make(map[string]string), - CreateTime: time.Now(), - StartTs: startTs, - TargetTs: targetTs, - Config: cfg, - Engine: model.SortEngine(sortEngine), - SortDir: sortDir, - State: model.StateNormal, + SinkURI: sinkURI, + Opts: make(map[string]string), + CreateTime: time.Now(), + StartTs: startTs, + TargetTs: targetTs, + Config: cfg, + Engine: model.SortEngine(sortEngine), + SortDir: sortDir, + State: model.StateNormal, + SyncPoint: syncPoint, + SyncInterval: syncInterval, } tz, err := util.GetTimezone(timezone) @@ -346,6 +348,8 @@ func newCreateChangefeedCommand() *cobra.Command { changefeedConfigVariables(command) command.PersistentFlags().BoolVar(&noConfirm, "no-confirm", false, "Don't ask user whether to ignore ineligible table") command.PersistentFlags().StringVarP(&changefeedID, "changefeed-id", "c", "", "Replication task (changefeed) ID") + command.PersistentFlags().BoolVar(&syncPoint, "sync-point", false, "(Expremental) Set and Record syncpoint in replication(default off)") + command.PersistentFlags().StringVar(&syncInterval, "sync-interval", "10m", "(Expremental) Set the interval for syncpoint in replication(default 10min)") return command } diff --git a/tests/syncpoint/deploy_config.go b/tests/syncpoint/deploy_config.go deleted file mode 100644 index 0717781c72b..00000000000 --- a/tests/syncpoint/deploy_config.go +++ /dev/null @@ -1,56 +0,0 @@ -// Copyright 2020 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 main - -import ( - "bytes" - "flag" - "fmt" - "os" - - "github.com/BurntSushi/toml" - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/util" -) - -func main() { - if len(os.Args) != 5 { - log.Info("wrong args,need three args!") - os.Exit(2) - } - primaryTs := os.Args[1] - secondaryTs := os.Args[2] - diffConfig := util.NewDiffConfig() - err := diffConfig.Parse(os.Args[3:]) - switch errors.Cause(err) { - case nil: - case flag.ErrHelp: - os.Exit(0) - default: - log.S().Errorf("parse cmd flags err %s\n", err) - os.Exit(2) - } - // if len(os.Args) != 3 { - // log.S().Errorf("wrong args,need two args %s\n", errors.New("wrong args")) - // os.Exit(2) - // } - diffConfig.SourceDBCfg[0].Snapshot = primaryTs - diffConfig.TargetDBCfg.Snapshot = secondaryTs - buf := new(bytes.Buffer) - if err := toml.NewEncoder(buf).Encode(diffConfig); err != nil { - log.Fatal("someting wrong") - } - fmt.Println(buf.String()) -} diff --git a/tests/syncpoint/run.sh b/tests/syncpoint/run.sh index 72a37c44803..2c969654ee2 100755 --- a/tests/syncpoint/run.sh +++ b/tests/syncpoint/run.sh @@ -97,7 +97,7 @@ function run() { SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" - run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" + run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --sync-point --sync-interval=10s goSql diff --git a/tests/syncpoint/test.sh b/tests/syncpoint/test.sh deleted file mode 100755 index 8c591fcdf3a..00000000000 --- a/tests/syncpoint/test.sh +++ /dev/null @@ -1,8 +0,0 @@ - -function deployConfig() { - cat ./conf/diff_config_part1.toml > ./conf/diff_config.toml - echo "snapshot = "$1"" >> ./conf/diff_config.toml - cat ./conf/diff_config_part2.toml >> ./conf/diff_config.toml - echo "snapshot = "$2"" >> ./conf/diff_config.toml -} -deployConfig 123 456 diff --git a/tests/util/diff_config.go b/tests/util/diff_config.go deleted file mode 100644 index 17ec90eeb30..00000000000 --- a/tests/util/diff_config.go +++ /dev/null @@ -1,371 +0,0 @@ -// Copyright 2020 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 util - -import ( - "database/sql" - "encoding/json" - "flag" - "net/url" - "strconv" - - "github.com/BurntSushi/toml" - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/parser/model" - "github.com/pingcap/tidb-tools/pkg/dbutil" - router "github.com/pingcap/tidb-tools/pkg/table-router" - "go.uber.org/zap" -) - -const ( - percent0 = 0 - percent100 = 100 -) - -var sourceInstanceMap map[string]interface{} = make(map[string]interface{}) - -// DBConfig is the config of database, and keep the connection. -type DiffDBConfig struct { - dbutil.DBConfig - - InstanceID string `toml:"instance-id" json:"instance-id"` - - Conn *sql.DB -} - -// Valid returns true if database's config is valide. -func (c *DiffDBConfig) Valid() bool { - if c.InstanceID == "" { - log.Error("must specify source database's instance id") - return false - } - sourceInstanceMap[c.InstanceID] = struct{}{} - - return true -} - -// CheckTables saves the tables need to check. -type DiffCheckTables struct { - // schema name - Schema string `toml:"schema" json:"schema"` - - // table list - Tables []string `toml:"tables" json:"tables"` - - ExcludeTables []string `toml:"exclude-tables" json:"exclude-tables"` -} - -// TableConfig is the config of table. -type DiffTableConfig struct { - // table's origin information - DiffTableInstance - // columns be ignored, will not check this column's data - IgnoreColumns []string `toml:"ignore-columns"` - // field should be the primary key, unique key or field with index - Fields string `toml:"index-fields"` - // select range, for example: "age > 10 AND age < 20" - Range string `toml:"range"` - // set true if comparing sharding tables with target table, should have more than one source tables. - IsSharding bool `toml:"is-sharding"` - // saves the source tables's info. - // may have more than one source for sharding tables. - // or you want to compare table with different schema and table name. - // SourceTables can be nil when source and target is one-to-one correspondence. - SourceTables []DiffTableInstance `toml:"source-tables"` - TargetTableInfo *model.TableInfo - - // collation config in mysql/tidb - Collation string `toml:"collation"` -} - -// Valid returns true if table's config is valide. -func (t *DiffTableConfig) Valid() bool { - if t.Schema == "" || t.Table == "" { - log.Error("schema and table's name can't be empty") - return false - } - - if t.IsSharding { - if len(t.SourceTables) <= 1 { - log.Error("must have more than one source tables if comparing sharding tables") - return false - } - - } else { - if len(t.SourceTables) > 1 { - log.Error("have more than one source table in no sharding mode") - return false - } - } - - for _, sourceTable := range t.SourceTables { - if !sourceTable.Valid() { - return false - } - } - - return true -} - -// TableInstance saves the base information of table. -type DiffTableInstance struct { - // database's instance id - InstanceID string `toml:"instance-id" json:"instance-id"` - // schema name - Schema string `toml:"schema"` - // table name - Table string `toml:"table"` -} - -// Valid returns true if table instance's info is valide. -// should be executed after source database's check. -func (t *DiffTableInstance) Valid() bool { - if t.InstanceID == "" { - log.Error("must specify the database's instance id for source table") - return false - } - - if _, ok := sourceInstanceMap[t.InstanceID]; !ok { - log.Error("unknown database instance id", zap.String("instance id", t.InstanceID)) - return false - } - - if t.Schema == "" || t.Table == "" { - log.Error("schema and table's name can't be empty") - return false - } - - return true -} - -// Config is the configuration. -type DiffConfig struct { - *flag.FlagSet `json:"-"` - - // log level - LogLevel string `toml:"log-level" json:"log-level"` - - // source database's config - SourceDBCfg []DiffDBConfig `toml:"source-db" json:"source-db"` - - // target database's config - TargetDBCfg DiffDBConfig `toml:"target-db" json:"target-db"` - - // for example, the whole data is [1...100] - // we can split these data to [1...10], [11...20], ..., [91...100] - // the [1...10] is a chunk, and it's chunk size is 10 - // size of the split chunk - ChunkSize int `toml:"chunk-size" json:"chunk-size"` - - // sampling check percent, for example 10 means only check 10% data - Sample int `toml:"sample-percent" json:"sample-percent"` - - // how many goroutines are created to check data - CheckThreadCount int `toml:"check-thread-count" json:"check-thread-count"` - - // set false if want to comapre the data directly - UseChecksum bool `toml:"use-checksum" json:"use-checksum"` - - // set true if just want compare data by checksum, will skip select data when checksum is not equal. - OnlyUseChecksum bool `toml:"only-use-checksum" json:"only-use-checksum"` - - // the name of the file which saves sqls used to fix different data - FixSQLFile string `toml:"fix-sql-file" json:"fix-sql-file"` - - // the tables to be checked - Tables []*DiffCheckTables `toml:"check-tables" json:"check-tables"` - - // TableRules defines table name and database name's conversion relationship between source database and target database - TableRules []*router.TableRule `toml:"table-rules" json:"table-rules"` - - // the config of table - TableCfgs []*DiffTableConfig `toml:"table-config" json:"table-config"` - - // ignore check table's struct - IgnoreStructCheck bool `toml:"ignore-struct-check" json:"ignore-struct-check"` - - // ignore check table's data - IgnoreDataCheck bool `toml:"ignore-data-check" json:"ignore-data-check"` - - // set true will continue check from the latest checkpoint - UseCheckpoint bool `toml:"use-checkpoint" json:"use-checkpoint"` - - // DMAddr is dm-master's address, the format should like "http://127.0.0.1:8261" - DMAddr string `toml:"dm-addr" json:"dm-addr"` - // DMTask is dm's task name - DMTask string `toml:"dm-task" json:"dm-task"` - - // config file - ConfigFile string - - // print version if set true - PrintVersion bool -} - -// NewConfig creates a new config. -func NewDiffConfig() *DiffConfig { - cfg := &DiffConfig{} - cfg.FlagSet = flag.NewFlagSet("diff", flag.ContinueOnError) - fs := cfg.FlagSet - - fs.StringVar(&cfg.ConfigFile, "config", "", "Config file") - fs.StringVar(&cfg.LogLevel, "L", "info", "log level: debug, info, warn, error, fatal") - fs.IntVar(&cfg.ChunkSize, "chunk-size", 1000, "diff check chunk size") - fs.IntVar(&cfg.Sample, "sample", 100, "the percent of sampling check") - fs.IntVar(&cfg.CheckThreadCount, "check-thread-count", 1, "how many goroutines are created to check data") - fs.BoolVar(&cfg.UseChecksum, "use-checksum", true, "set false if want to comapre the data directly") - fs.StringVar(&cfg.FixSQLFile, "fix-sql-file", "fix.sql", "the name of the file which saves sqls used to fix different data") - fs.BoolVar(&cfg.PrintVersion, "V", false, "print version of sync_diff_inspector") - fs.BoolVar(&cfg.IgnoreDataCheck, "ignore-data-check", false, "ignore check table's data") - fs.BoolVar(&cfg.IgnoreStructCheck, "ignore-struct-check", false, "ignore check table's struct") - fs.BoolVar(&cfg.UseCheckpoint, "use-checkpoint", true, "set true will continue check from the latest checkpoint") - - return cfg -} - -// Parse parses flag definitions from the argument list. -func (c *DiffConfig) Parse(arguments []string) error { - // Parse first to get config file. - err := c.FlagSet.Parse(arguments) - if err != nil { - return errors.Trace(err) - } - - // Load config file if specified. - if c.ConfigFile != "" { - err = c.configFromFile(c.ConfigFile) - if err != nil { - return errors.Trace(err) - } - } - - // Parse again to replace with command line options. - err = c.FlagSet.Parse(arguments) - if err != nil { - return errors.Trace(err) - } - - if len(c.FlagSet.Args()) != 0 { - return errors.Errorf("'%s' is an invalid flag", c.FlagSet.Arg(0)) - } - - return nil -} - -func (c *DiffConfig) String() string { - cfg, err := json.Marshal(c) - if err != nil { - return "" - } - return string(cfg) -} - -// configFromFile loads config from file. -func (c *DiffConfig) configFromFile(path string) error { - _, err := toml.DecodeFile(path, c) - return errors.Trace(err) -} - -func (c *DiffConfig) configToFile(path string) error { - _, err := toml.DecodeFile(path, c) - return errors.Trace(err) -} - -func (c *DiffConfig) checkConfig() bool { - if c.Sample > percent100 || c.Sample < percent0 { - log.Error("sample must be greater than 0 and less than or equal to 100!") - return false - } - - if c.CheckThreadCount <= 0 { - log.Error("check-thcount must greater than 0!") - return false - } - - if len(c.DMAddr) != 0 { - u, err := url.Parse(c.DMAddr) - if err != nil || u.Scheme == "" || u.Host == "" { - log.Error("dm-addr's format should like 'http://127.0.0.1:8261'") - return false - } - - if len(c.DMTask) == 0 { - log.Error("must set the `dm-task` if set `dm-addr`") - return false - } - - emptyDBConfig := DiffDBConfig{} - // source DB, target DB and check table's information will get from DM, should not set them - if len(c.SourceDBCfg) != 0 || c.TargetDBCfg != emptyDBConfig { - log.Error("should not set `source-db` or `target-db`, diff will generate them automatically when set `dm-addr` and `dm-task`") - return false - } - - if len(c.Tables) != 0 || len(c.TableRules) != 0 || len(c.TableCfgs) != 0 { - log.Error("should not set `check-tables`, `table-rules` or `table-config`, diff will generate them automatically when set `dm-addr` and `dm-task`") - return false - } - } else { - if len(c.SourceDBCfg) == 0 { - log.Error("must have at least one source database") - return false - } - - for i := range c.SourceDBCfg { - if !c.SourceDBCfg[i].Valid() { - return false - } - if c.SourceDBCfg[i].Snapshot != "" { - c.SourceDBCfg[i].Snapshot = strconv.Quote(c.SourceDBCfg[i].Snapshot) - } - } - - if c.TargetDBCfg.InstanceID == "" { - c.TargetDBCfg.InstanceID = "target" - } - if c.TargetDBCfg.Snapshot != "" { - c.TargetDBCfg.Snapshot = strconv.Quote(c.TargetDBCfg.Snapshot) - } - if _, ok := sourceInstanceMap[c.TargetDBCfg.InstanceID]; ok { - log.Error("target has same instance id in source", zap.String("instance id", c.TargetDBCfg.InstanceID)) - return false - } - - if len(c.Tables) == 0 { - log.Error("must specify check tables") - return false - } - - for _, tableCfg := range c.TableCfgs { - if !tableCfg.Valid() { - return false - } - } - } - - if c.OnlyUseChecksum { - if !c.UseChecksum { - log.Error("need set use-checksum = true") - return false - } - } else { - if len(c.FixSQLFile) == 0 { - log.Warn("fix-sql-file is invalid, will use default value 'fix.sql'") - c.FixSQLFile = "fix.sql" - } - } - - return true -} From 0a83609d290100b82d3085ffe6a8c6d6610c8231 Mon Sep 17 00:00:00 2001 From: colin Date: Mon, 14 Sep 2020 02:12:49 +0800 Subject: [PATCH 12/35] add some flags in 'cdc cli changefeed' --- Dockerfile | 1 - cdc/changefeed.go | 45 +++++++++++++++++++++++++++++++++------- cdc/owner.go | 21 ++++++++++++++++--- cmd/client_changefeed.go | 4 ++-- 4 files changed, 58 insertions(+), 13 deletions(-) diff --git a/Dockerfile b/Dockerfile index b76c609b9b9..b04ec56945d 100644 --- a/Dockerfile +++ b/Dockerfile @@ -2,7 +2,6 @@ FROM golang:1.14-alpine as builder RUN apk add --no-cache git make bash WORKDIR /go/src/github.com/pingcap/ticdc COPY . . -RUN go env -w GO111MODULE=on && go env -w GOPROXY=https://goproxy.cn,direct RUN make FROM alpine:3.12 diff --git a/cdc/changefeed.go b/cdc/changefeed.go index f6a276b9890..aa406c3e0ad 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -87,6 +87,7 @@ type changeFeed struct { updateResolvedTs bool startTimer chan bool syncDB *sql.DB + syncCancel context.CancelFunc taskStatus model.ProcessorsInfos taskPositions map[model.CaptureID]*model.TaskPosition filter *filter.Filter @@ -722,8 +723,6 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { if c.info.SyncPoint { //if (c.status.ResolvedTs == c.status.CheckpointTs && (c.updateResolvedTs == false || c.status.ResolvedTs == c.ddlResolvedTs)) || c.status.ResolvedTs == 0 { if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { - log.Debug("achive the sync point", zap.Uint64("ResolvedTs", c.status.ResolvedTs)) - //c.syncPointTs = c.targetTs //恢复syncPointTs,使得ResoledTs可以继续推进 log.Info("achive the sync point with timer", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) c.updateResolvedTs = true c.sinkSyncpoint(ctx) @@ -906,12 +905,31 @@ func (c *changeFeed) startSyncPeriod(ctx context.Context, interval time.Duration } //createSynctable create a sync table to record the -func (c *changeFeed) createSynctable() { +func (c *changeFeed) createSynctable(ctx context.Context) error { database := "TiCDC" - c.syncDB.Exec("CREATE DATABASE IF NOT EXISTS " + database) - c.syncDB.Exec("USE " + database) - c.syncDB.Exec("CREATE TABLE IF NOT EXISTS syncpoint (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `primary_ts` ) )") - // todo err 处理 + tx, err := c.syncDB.BeginTx(ctx, nil) + if err != nil { + log.Info("create sync table: begin Tx fail") + return err + } + _, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database) + //_,err := c.syncDB.Exec("CREATE DATABASE IF NOT EXISTS " + database) + if err != nil { + tx.Rollback() + return err + } + _, err = tx.Exec("USE " + database) + if err != nil { + tx.Rollback() + return err + } + _, err = tx.Exec("CREATE TABLE IF NOT EXISTS syncpoint (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `primary_ts` ) )") + if err != nil { + tx.Rollback() + return err + } + err = tx.Commit() + return err } //sinkSyncpoint record the syncpoint(a map with ts) in downstream db @@ -934,3 +952,16 @@ func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { tx.Commit() //TODO deal with error return nil } + +func (c *changeFeed) stopSyncPointTicker() { + if c.syncCancel != nil { + c.syncCancel() + c.syncCancel = nil + } +} + +func (c *changeFeed) startSyncPointTicker(ctx context.Context, interval time.Duration) { + var syncCtx context.Context + syncCtx, c.syncCancel = context.WithCancel(ctx) + c.startSyncPeriod(syncCtx, interval) +} diff --git a/cdc/owner.go b/cdc/owner.go index a795b6a2ce3..47eca2e6b37 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -353,6 +353,7 @@ func (o *Owner) newChangeFeed( updateResolvedTs: true, startTimer: make(chan bool), syncDB: syncDB, + syncCancel: nil, taskStatus: processorsInfos, taskPositions: taskPositions, etcdCli: o.etcdClient, @@ -502,21 +503,26 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { zap.String("changefeed", changeFeedID), zap.Error(err)) continue } - o.changeFeeds[changeFeedID] = newCf + if newCf.info.SyncPoint { log.Info("syncpoint is on", zap.Bool("syncpoint", newCf.info.SyncPoint)) //create the sync table - newCf.createSynctable() + err := newCf.createSynctable(ctx) + if err != nil { + return err + } + //start SyncPeriod for create the sync point syncInterval, err := time.ParseDuration(newCf.info.SyncInterval) if err != nil { syncInterval = defaultSyncInterval } - newCf.startSyncPeriod(ctx, syncInterval) + newCf.startSyncPointTicker(ctx, syncInterval) } else { log.Info("syncpoint is off", zap.Bool("syncpoint", newCf.info.SyncPoint)) } + o.changeFeeds[changeFeedID] = newCf delete(o.stoppedFeeds, changeFeedID) } o.adminJobsLock.Lock() @@ -798,6 +804,7 @@ func (o *Owner) handleAdminJob(ctx context.Context) error { if err != nil { return errors.Trace(err) } + cf.stopSyncPointTicker() case model.AdminRemove, model.AdminFinish: if cf != nil { err := o.dispatchJob(ctx, job) @@ -832,6 +839,7 @@ func (o *Owner) handleAdminJob(ctx context.Context) error { if err != nil { return errors.Trace(err) } + cf.stopSyncPointTicker() case model.AdminResume: // resume changefeed must read checkpoint from ChangeFeedStatus if errors.Cause(err) == model.ErrChangeFeedNotExists { @@ -863,6 +871,13 @@ func (o *Owner) handleAdminJob(ctx context.Context) error { if err != nil { return errors.Trace(err) } + if cf.info.SyncPoint { + syncInterval, err := time.ParseDuration(cf.info.SyncInterval) + if err != nil { + syncInterval = defaultSyncInterval + } + cf.startSyncPointTicker(ctx, syncInterval) + } } // TODO: we need a better admin job workflow. Supposing uses create // multiple admin jobs to a specific changefeed at the same time, such diff --git a/cmd/client_changefeed.go b/cmd/client_changefeed.go index 78c5466d0ab..9d67919f2b7 100644 --- a/cmd/client_changefeed.go +++ b/cmd/client_changefeed.go @@ -311,6 +311,8 @@ func changefeedConfigVariables(command *cobra.Command) { command.PersistentFlags().Uint64Var(&cyclicReplicaID, "cyclic-replica-id", 0, "(Expremental) Cyclic replication replica ID of changefeed") command.PersistentFlags().UintSliceVar(&cyclicFilterReplicaIDs, "cyclic-filter-replica-ids", []uint{}, "(Expremental) Cyclic replication filter replica ID of changefeed") command.PersistentFlags().BoolVar(&cyclicSyncDDL, "cyclic-sync-ddl", true, "(Expremental) Cyclic replication sync DDL of changefeed") + command.PersistentFlags().BoolVar(&syncPoint, "sync-point", false, "(Expremental) Set and Record syncpoint in replication(default off)") + command.PersistentFlags().StringVar(&syncInterval, "sync-interval", "10m", "(Expremental) Set the interval for syncpoint in replication(default 10min)") } func newCreateChangefeedCommand() *cobra.Command { @@ -348,8 +350,6 @@ func newCreateChangefeedCommand() *cobra.Command { changefeedConfigVariables(command) command.PersistentFlags().BoolVar(&noConfirm, "no-confirm", false, "Don't ask user whether to ignore ineligible table") command.PersistentFlags().StringVarP(&changefeedID, "changefeed-id", "c", "", "Replication task (changefeed) ID") - command.PersistentFlags().BoolVar(&syncPoint, "sync-point", false, "(Expremental) Set and Record syncpoint in replication(default off)") - command.PersistentFlags().StringVar(&syncInterval, "sync-interval", "10m", "(Expremental) Set the interval for syncpoint in replication(default 10min)") return command } From 1da2a73eb73a105d73d3ca320ddad756f0fc603b Mon Sep 17 00:00:00 2001 From: colin Date: Mon, 14 Sep 2020 10:51:08 +0800 Subject: [PATCH 13/35] fix the changefeed resume logic --- cdc/owner.go | 7 ------- 1 file changed, 7 deletions(-) diff --git a/cdc/owner.go b/cdc/owner.go index 47eca2e6b37..771591973d9 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -871,13 +871,6 @@ func (o *Owner) handleAdminJob(ctx context.Context) error { if err != nil { return errors.Trace(err) } - if cf.info.SyncPoint { - syncInterval, err := time.ParseDuration(cf.info.SyncInterval) - if err != nil { - syncInterval = defaultSyncInterval - } - cf.startSyncPointTicker(ctx, syncInterval) - } } // TODO: we need a better admin job workflow. Supposing uses create // multiple admin jobs to a specific changefeed at the same time, such From b1dad1d811ad959fff573e60c8dfa556b6be1567 Mon Sep 17 00:00:00 2001 From: colin Date: Tue, 15 Sep 2020 14:49:15 +0800 Subject: [PATCH 14/35] merge master and fix some err --- cdc/changefeed.go | 2 +- cdc/owner.go | 4 ++-- cdc/sink/mysql.go | 15 ++++++++++++++- tests/syncpoint/run.sh | 2 +- 4 files changed, 18 insertions(+), 5 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 33cb777bb49..6599a902e84 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -889,7 +889,6 @@ func (c *changeFeed) pullDDLJob() error { return nil } - // startSyncPeriod start a timer for every changefeed to create sync point by time func (c *changeFeed) startSyncPeriod(ctx context.Context, interval time.Duration) { log.Debug("sync ticker start", zap.Duration("sync-interval", interval)) @@ -966,6 +965,7 @@ func (c *changeFeed) startSyncPointTicker(ctx context.Context, interval time.Dur var syncCtx context.Context syncCtx, c.syncCancel = context.WithCancel(ctx) c.startSyncPeriod(syncCtx, interval) +} func (c *changeFeed) Close() { err := c.ddlHandler.Close() diff --git a/cdc/owner.go b/cdc/owner.go index 8981a329741..b9f46046fc1 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -338,8 +338,8 @@ func (o *Owner) newChangeFeed( return nil, errors.Trace(err) } defer func() { - if resultErr != nil && sink != nil { - sink.Close() + if resultErr != nil && primarySink != nil { + primarySink.Close() } }() go func() { diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 054fa90dbc5..a2597ddef56 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -1064,7 +1064,20 @@ func NewSyncpointSinklink(ctx context.Context, info *model.ChangeFeedInfo, id st if err != nil { return nil, errors.Trace(err) } - dsnStr, err = configureSinkURI(ctx, dsn, util.TimezoneFromCtx(ctx), params) + + tz := util.TimezoneFromCtx(ctx) + // create test db used for parameter detection + if dsn.Params == nil { + dsn.Params = make(map[string]string, 1) + } + dsn.Params["time_zone"] = fmt.Sprintf(`"%s"`, tz.String()) + testDB, err := sql.Open("mysql", dsn.FormatDSN()) + if err != nil { + return nil, errors.Annotate( + cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection when configuring sink") + } + defer testDB.Close() + dsnStr, err = configureSinkURI(ctx, dsn, tz, params, testDB) if err != nil { return nil, errors.Trace(err) } diff --git a/tests/syncpoint/run.sh b/tests/syncpoint/run.sh index 2c969654ee2..f8a3a8fc35f 100755 --- a/tests/syncpoint/run.sh +++ b/tests/syncpoint/run.sh @@ -91,7 +91,7 @@ function run() { cd $WORK_DIR - start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST:$UP_PD_PORT) + start_ts=$(run_cdc_cli tso query --pd=http://$UP_PD_HOST_1:$UP_PD_PORT_1) run_sql "CREATE DATABASE testSync;" run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY From 7e2ca72d0ba8572f015d4504edd55558034dd2a5 Mon Sep 17 00:00:00 2001 From: colin Date: Tue, 15 Sep 2020 15:32:23 +0800 Subject: [PATCH 15/35] fix some err about sql for ci-lint --- cdc/changefeed.go | 47 ++++++++++++++++++++++++++++++++++++----------- 1 file changed, 36 insertions(+), 11 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 6599a902e84..d887d12830d 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -726,7 +726,10 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { log.Info("achive the sync point with timer", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) c.updateResolvedTs = true - c.sinkSyncpoint(ctx) + err := c.sinkSyncpoint(ctx) + if err != nil { + log.Error("syncpoint sinl fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs)) + } } if c.status.ResolvedTs == 0 { @@ -735,7 +738,10 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs { log.Info("achive the sync point with ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) - c.sinkSyncpoint(ctx) + err := c.sinkSyncpoint(ctx) + if err != nil { + log.Error("syncpoint sinl fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs)) + } c.ddlTs = 0 } } @@ -916,17 +922,26 @@ func (c *changeFeed) createSynctable(ctx context.Context) error { _, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database) //_,err := c.syncDB.Exec("CREATE DATABASE IF NOT EXISTS " + database) if err != nil { - tx.Rollback() + err2 := tx.Rollback() + if err2 != nil { + log.Error(err2.Error()) + } return err } _, err = tx.Exec("USE " + database) if err != nil { - tx.Rollback() + err2 := tx.Rollback() + if err2 != nil { + log.Error(err2.Error()) + } return err } _, err = tx.Exec("CREATE TABLE IF NOT EXISTS syncpoint (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `primary_ts` ) )") if err != nil { - tx.Rollback() + err2 := tx.Rollback() + if err2 != nil { + log.Error(err2.Error()) + } return err } err = tx.Commit() @@ -941,17 +956,27 @@ func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { return err } row := tx.QueryRow("select @@tidb_current_ts") - var slaveTs string - err = row.Scan(&slaveTs) + var secondaryTs string + err = row.Scan(&secondaryTs) if err != nil { log.Info("sync table: get tidb_current_ts err") - tx.Rollback() + err2 := tx.Rollback() + if err2 != nil { + log.Error(err2.Error()) + } + return err + } + _, err = tx.Exec("insert into TiCDC.syncpoint(cf, primary_ts, secondary_ts) VALUES (?,?,?)", c.id, c.status.CheckpointTs, secondaryTs) + if err != nil { + err2 := tx.Rollback() + if err2 != nil { + log.Error(err2.Error()) + } return err } - tx.Exec("insert into TiCDC.syncpoint(cf, primary_ts, secondary_ts) VALUES (?,?,?)", c.id, c.status.CheckpointTs, slaveTs) //tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, 0) - tx.Commit() //TODO deal with error - return nil + err = tx.Commit() //TODO deal with error + return err } func (c *changeFeed) stopSyncPointTicker() { From 22fca2a1f4b71c58d7e394c993d43fbe686033cc Mon Sep 17 00:00:00 2001 From: colin Date: Tue, 15 Sep 2020 15:47:51 +0800 Subject: [PATCH 16/35] fix code fmt --- cmd/client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/client.go b/cmd/client.go index 3149629e63f..2f026eaa309 100644 --- a/cmd/client.go +++ b/cmd/client.go @@ -76,7 +76,7 @@ var ( syncPoint bool syncInterval string - + optForceRemove bool defaultContext context.Context From 85d1482631f8df0f2bc9527946919ff39f95f53b Mon Sep 17 00:00:00 2001 From: colin Date: Wed, 16 Sep 2020 21:54:17 +0800 Subject: [PATCH 17/35] fix the test script to run all integration test by one command --- Makefile | 4 ++-- tests/run.sh | 7 +++++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/Makefile b/Makefile index b21a10bac6a..a56a7fce176 100644 --- a/Makefile +++ b/Makefile @@ -103,10 +103,10 @@ integration_test_build: check_failpoint_ctl integration_test: integration_test_mysql integration_test_mysql: check_third_party_binary - tests/run.sh $(CASE) mysql + tests/run.sh mysql $(CASE) integration_test_kafka: check_third_party_binary - tests/run.sh $(CASE) kafka + tests/run.sh kafka $(CASE) fmt: @echo "gofmt (simplify)" diff --git a/tests/run.sh b/tests/run.sh index 3f4cb240e57..9e3928c12c4 100755 --- a/tests/run.sh +++ b/tests/run.sh @@ -41,12 +41,15 @@ run_case() { bash "$script" "$sink_type" } -test_case=$1 -sink_type=$2 +sink_type=$1 + +set +eu +test_case=$2 if [ -z "$test_case" ]; then test_case="*" fi +set -eu if [ "$test_case" == "*" ]; then for script in $CUR/*/run.sh; do From 2563a394cea9fe9a723a443d874e19de42d9642d Mon Sep 17 00:00:00 2001 From: colin Date: Thu, 17 Sep 2020 12:35:00 +0800 Subject: [PATCH 18/35] fix the nil pointer err in owner.go --- README.md | 2 -- cdc/changefeed.go | 24 ++++++++++----------- cdc/owner.go | 47 +----------------------------------------- tests/syncpoint/run.sh | 2 ++ 4 files changed, 14 insertions(+), 61 deletions(-) diff --git a/README.md b/README.md index 15da4afa526..dfb5f29e348 100644 --- a/README.md +++ b/README.md @@ -68,5 +68,3 @@ for details on submitting patches and the contribution workflow. ## License TiCDC is under the Apache 2.0 license. See the [LICENSE](./LICENSE) file for details. - -## Branch For dev diff --git a/cdc/changefeed.go b/cdc/changefeed.go index d887d12830d..37e39f17a38 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -917,16 +917,15 @@ func (c *changeFeed) createSynctable(ctx context.Context) error { tx, err := c.syncDB.BeginTx(ctx, nil) if err != nil { log.Info("create sync table: begin Tx fail") - return err + return cerror.WrapError(cerror.ErrMySQLTxnError, err) } _, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database) - //_,err := c.syncDB.Exec("CREATE DATABASE IF NOT EXISTS " + database) if err != nil { err2 := tx.Rollback() if err2 != nil { log.Error(err2.Error()) } - return err + return cerror.WrapError(cerror.ErrMySQLTxnError, err) } _, err = tx.Exec("USE " + database) if err != nil { @@ -934,18 +933,18 @@ func (c *changeFeed) createSynctable(ctx context.Context) error { if err2 != nil { log.Error(err2.Error()) } - return err + return cerror.WrapError(cerror.ErrMySQLTxnError, err) } - _, err = tx.Exec("CREATE TABLE IF NOT EXISTS syncpoint (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `primary_ts` ) )") + _, err = tx.Exec("CREATE TABLE IF NOT EXISTS syncpoint (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `cf`, `primary_ts` ) )") if err != nil { err2 := tx.Rollback() if err2 != nil { log.Error(err2.Error()) } - return err + return cerror.WrapError(cerror.ErrMySQLTxnError, err) } err = tx.Commit() - return err + return cerror.WrapError(cerror.ErrMySQLTxnError, err) } //sinkSyncpoint record the syncpoint(a map with ts) in downstream db @@ -953,7 +952,7 @@ func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { tx, err := c.syncDB.BeginTx(ctx, nil) if err != nil { log.Info("sync table: begin Tx fail") - return err + return cerror.WrapError(cerror.ErrMySQLTxnError, err) } row := tx.QueryRow("select @@tidb_current_ts") var secondaryTs string @@ -964,7 +963,7 @@ func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { if err2 != nil { log.Error(err2.Error()) } - return err + return cerror.WrapError(cerror.ErrMySQLTxnError, err) } _, err = tx.Exec("insert into TiCDC.syncpoint(cf, primary_ts, secondary_ts) VALUES (?,?,?)", c.id, c.status.CheckpointTs, secondaryTs) if err != nil { @@ -972,11 +971,10 @@ func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { if err2 != nil { log.Error(err2.Error()) } - return err + return cerror.WrapError(cerror.ErrMySQLTxnError, err) } - //tx.Exec("insert into TiCDC.syncpoint( master_ts, slave_ts) VALUES (?,?)", c.status.CheckpointTs, 0) - err = tx.Commit() //TODO deal with error - return err + err = tx.Commit() + return cerror.WrapError(cerror.ErrMySQLTxnError, err) } func (c *changeFeed) stopSyncPointTicker() { diff --git a/cdc/owner.go b/cdc/owner.go index b9f46046fc1..5cd41ee6ab5 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -836,6 +836,7 @@ func (o *Owner) handleAdminJob(ctx context.Context) error { cf.stopSyncPointTicker() case model.AdminRemove, model.AdminFinish: if cf != nil { + cf.stopSyncPointTicker() err := o.dispatchJob(ctx, job) if err != nil { return errors.Trace(err) @@ -883,7 +884,6 @@ func (o *Owner) handleAdminJob(ctx context.Context) error { return errors.Trace(err) } } - cf.stopSyncPointTicker() case model.AdminResume: // resume changefeed must read checkpoint from ChangeFeedStatus if cerror.ErrChangeFeedNotExists.Equal(err) { @@ -1326,48 +1326,3 @@ func (o *Owner) startCaptureWatcher(ctx context.Context) { } }() } - -/*func (o *Owner) configureSinkURI(ctx context.Context, dsnCfg *dmysql.Config, tz *time.Location) (string, error) { -if dsnCfg.Params == nil { - dsnCfg.Params = make(map[string]string, 1) -} -dsnCfg.DBName = "" -dsnCfg.InterpolateParams = true -dsnCfg.MultiStatements = true -dsnCfg.Params["time_zone"] = fmt.Sprintf(`"%s"`, tz.String()) - -testDB, err := sql.Open("mysql", dsnCfg.FormatDSN()) -if err != nil { - return "", errors.Annotate(err, "fail to open MySQL connection when configuring sink") -} -defer testDB.Close() -log.Debug("Opened connection to configure some tidb special parameters") - -var variableName string -var autoRandomInsertEnabled string -queryStr := "show session variables like 'allow_auto_random_explicit_insert';" -err = testDB.QueryRowContext(ctx, queryStr).Scan(&variableName, &autoRandomInsertEnabled) -if err != nil && err != sql.ErrNoRows { - return "", errors.Annotate(err, "fail to query sink for support of auto-random") -} -if err == nil && (autoRandomInsertEnabled == "off" || autoRandomInsertEnabled == "0") { - dsnCfg.Params["allow_auto_random_explicit_insert"] = "1" - log.Debug("Set allow_auto_random_explicit_insert to 1") -} - -var txnMode string -queryStr = "show session variables like 'tidb_txn_mode';" -err = testDB.QueryRowContext(ctx, queryStr).Scan(&variableName, &txnMode) -if err != nil && err != sql.ErrNoRows { - return "", errors.Annotate(err, "fail to query sink for txn mode") -} -/*if err == nil { - dsnCfg.Params["tidb_txn_mode"] = params.tidbTxnMode -}*/ - -/* dsnClone := dsnCfg.Clone() - dsnClone.Passwd = "******" - log.Info("sink uri is configured", zap.String("format dsn", dsnClone.FormatDSN())) - - return dsnCfg.FormatDSN(), nil -}*/ diff --git a/tests/syncpoint/run.sh b/tests/syncpoint/run.sh index f8a3a8fc35f..09b9ff0e134 100755 --- a/tests/syncpoint/run.sh +++ b/tests/syncpoint/run.sh @@ -11,6 +11,8 @@ SINK_TYPE=$1 CDC_COUNT=3 DB_COUNT=4 +# The follow `sleep 2` make sure the ddl and dml operation can always execute during the test. +# So every syncpoint will have different data and the diff tool can judge the syncpoint's validity. function ddl() { run_sql "DROP table IF EXISTS testSync.simple1" sleep 2 From 17de49258abec6128141359ced551f74c6dd4ea1 Mon Sep 17 00:00:00 2001 From: colin Date: Thu, 17 Sep 2020 22:05:36 +0800 Subject: [PATCH 19/35] fix the logic of deal with NewSyncpointSinklink's err in owner.go and use Duration as flag syncInterval in client_changefeed.go --- cdc/model/changefeed.go | 4 ++-- cdc/owner.go | 10 ++-------- cmd/client.go | 2 +- cmd/client_changefeed.go | 2 +- scripts/fix_lib_zstd.sh | 1 - 5 files changed, 6 insertions(+), 13 deletions(-) diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index c7340a23d43..fc13489a558 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -82,8 +82,8 @@ type ChangeFeedInfo struct { ErrorHis []int64 `json:"history"` Error *RunningError `json:"error"` - SyncPoint bool `json:"sync-point"` - SyncInterval string `json:"sync-interval"` + SyncPoint bool `json:"sync-point"` + SyncInterval time.Duration `json:"sync-interval"` } var changeFeedIDRe *regexp.Regexp = regexp.MustCompile(`^[a-zA-Z0-9]+(\-[a-zA-Z0-9]+)*$`) diff --git a/cdc/owner.go b/cdc/owner.go index 5cd41ee6ab5..06f5c673aa3 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -359,7 +359,7 @@ func (o *Owner) newChangeFeed( syncDB, err := sink.NewSyncpointSinklink(ctx, info, id) if err != nil { - log.Error("error on running owner", zap.Error(err)) + return nil, errors.Trace(err) } cf = &changeFeed{ @@ -541,13 +541,7 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { if err != nil { return err } - - //start SyncPeriod for create the sync point - syncInterval, err := time.ParseDuration(newCf.info.SyncInterval) - if err != nil { - syncInterval = defaultSyncInterval - } - newCf.startSyncPointTicker(ctx, syncInterval) + newCf.startSyncPointTicker(ctx, newCf.info.SyncInterval) } else { log.Info("syncpoint is off", zap.Bool("syncpoint", newCf.info.SyncPoint)) } diff --git a/cmd/client.go b/cmd/client.go index 2f026eaa309..dd144b7fbf9 100644 --- a/cmd/client.go +++ b/cmd/client.go @@ -75,7 +75,7 @@ var ( interval uint syncPoint bool - syncInterval string + syncInterval time.Duration optForceRemove bool diff --git a/cmd/client_changefeed.go b/cmd/client_changefeed.go index b65db8fd5b5..83c773584b3 100644 --- a/cmd/client_changefeed.go +++ b/cmd/client_changefeed.go @@ -344,7 +344,7 @@ func changefeedConfigVariables(command *cobra.Command) { command.PersistentFlags().UintSliceVar(&cyclicFilterReplicaIDs, "cyclic-filter-replica-ids", []uint{}, "(Expremental) Cyclic replication filter replica ID of changefeed") command.PersistentFlags().BoolVar(&cyclicSyncDDL, "cyclic-sync-ddl", true, "(Expremental) Cyclic replication sync DDL of changefeed") command.PersistentFlags().BoolVar(&syncPoint, "sync-point", false, "(Expremental) Set and Record syncpoint in replication(default off)") - command.PersistentFlags().StringVar(&syncInterval, "sync-interval", "10m", "(Expremental) Set the interval for syncpoint in replication(default 10min)") + command.PersistentFlags().DurationVar(&syncInterval, "sync-interval", 10*time.Minute, "(Expremental) Set the interval for syncpoint in replication(default 10min)") } func newCreateChangefeedCommand() *cobra.Command { diff --git a/scripts/fix_lib_zstd.sh b/scripts/fix_lib_zstd.sh index eec0d60bce0..9df1fb2e4a4 100755 --- a/scripts/fix_lib_zstd.sh +++ b/scripts/fix_lib_zstd.sh @@ -3,7 +3,6 @@ user=$(whoami) GOOS=$(go env GOOS) GOARCH=$(go env GOARCH) -GOPATH=$(go env GOPATH) module="github.com/valyala/gozstd@v1.7.0" GO111MODULE=on go mod download ${module} From 8a16cbd50d2a41b49d5170d30fff005791d4352a Mon Sep 17 00:00:00 2001 From: colin Date: Thu, 17 Sep 2020 22:16:34 +0800 Subject: [PATCH 20/35] remove unused var defaultSyncInterval --- cdc/changefeed.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 37e39f17a38..974e1d88585 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -36,8 +36,6 @@ import ( "go.uber.org/zap" ) -const defaultSyncInterval = time.Minute * 10 //default Interval for record syncpoint - type tableIDMap = map[model.TableID]struct{} // OwnerDDLHandler defines the ddl handler for Owner From fb8e6cc0ca0e66453518fbf30be4846266804adb Mon Sep 17 00:00:00 2001 From: colin Date: Fri, 18 Sep 2020 15:15:33 +0800 Subject: [PATCH 21/35] fix sink's scheme identification in mysql.go --- cdc/sink/mysql.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index a2597ddef56..5ea0f01bacb 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -1016,7 +1016,7 @@ func NewSyncpointSinklink(ctx context.Context, info *model.ChangeFeedInfo, id st } //todo If is neither mysql nor tidb, such as kafka, just ignore this feature. scheme := strings.ToLower(sinkURI.Scheme) - if scheme != "mysql" && scheme != "tidb" { + if scheme != "mysql" && scheme != "tidb" && scheme != "mysql+ssl" && scheme != "tidb+ssl" { return nil, errors.New("can create mysql sink with unsupported scheme") } params := defaultParams From a90718e474288fbf94378587bc45a8fe26ec1e5b Mon Sep 17 00:00:00 2001 From: colin Date: Fri, 18 Sep 2020 15:47:38 +0800 Subject: [PATCH 22/35] fix data race for c.updateResolvedTs in changefeed.go --- cdc/changefeed.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 974e1d88585..3f05157451c 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -18,6 +18,7 @@ import ( "database/sql" "fmt" "math" + "sync" "time" "github.com/pingcap/errors" @@ -83,6 +84,7 @@ type changeFeed struct { ddlState model.ChangeFeedDDLState targetTs uint64 ddlTs uint64 + syncpointMutex sync.Mutex updateResolvedTs bool startTimer chan bool syncDB *sql.DB @@ -720,6 +722,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { //sync-point on if c.info.SyncPoint { + c.syncpointMutex.Lock() //if (c.status.ResolvedTs == c.status.CheckpointTs && (c.updateResolvedTs == false || c.status.ResolvedTs == c.ddlResolvedTs)) || c.status.ResolvedTs == 0 { if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { log.Info("achive the sync point with timer", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) @@ -849,6 +852,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { c.startTimer <- true }*/ } + c.syncpointMutex.Unlock() } else if minResolvedTs > c.status.ResolvedTs { c.status.ResolvedTs = minResolvedTs tsUpdated = true @@ -903,7 +907,9 @@ func (c *changeFeed) startSyncPeriod(ctx context.Context, interval time.Duration case <-ctx.Done(): return case <-ticker.C: + c.syncpointMutex.Lock() c.updateResolvedTs = false + c.syncpointMutex.Unlock() } } }(ctx) From aeb7eb54c416be167f750d74ddcab11ae4bf4daf Mon Sep 17 00:00:00 2001 From: colin Date: Fri, 18 Sep 2020 16:14:51 +0800 Subject: [PATCH 23/35] fix some log's info in changefeed.go --- cdc/changefeed.go | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 3f05157451c..6ee80ffabc3 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -723,13 +723,12 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { //sync-point on if c.info.SyncPoint { c.syncpointMutex.Lock() - //if (c.status.ResolvedTs == c.status.CheckpointTs && (c.updateResolvedTs == false || c.status.ResolvedTs == c.ddlResolvedTs)) || c.status.ResolvedTs == 0 { if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { - log.Info("achive the sync point with timer", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) + log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) c.updateResolvedTs = true err := c.sinkSyncpoint(ctx) if err != nil { - log.Error("syncpoint sinl fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs)) + log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs)) } } @@ -738,10 +737,10 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { } if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs { - log.Info("achive the sync point with ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) + log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) err := c.sinkSyncpoint(ctx) if err != nil { - log.Error("syncpoint sinl fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs)) + log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs)) } c.ddlTs = 0 } @@ -817,7 +816,6 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { if minResolvedTs > c.ddlResolvedTs { minResolvedTs = c.ddlResolvedTs - //c.ddlTs = c.ddlResolvedTs } } @@ -843,14 +841,8 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { //syncpoint on if c.info.SyncPoint { if c.updateResolvedTs && minResolvedTs > c.status.ResolvedTs { - //prevResolvedTs := c.status.ResolvedTs c.status.ResolvedTs = minResolvedTs tsUpdated = true - /*if prevResolvedTs == c.status.CheckpointTs || prevResolvedTs == 0 { - //到达sync point - //todo 需要重新开始启动计时 - c.startTimer <- true - }*/ } c.syncpointMutex.Unlock() } else if minResolvedTs > c.status.ResolvedTs { From 872c029dc9a005655af550731ed233a2fcab2e35 Mon Sep 17 00:00:00 2001 From: colin Date: Fri, 18 Sep 2020 16:22:30 +0800 Subject: [PATCH 24/35] change 'syncPoint' -> 'syncPointEnabled' and 'syncInterval' -> 'syncPointInterval' for readability --- cdc/changefeed.go | 4 ++-- cdc/model/changefeed.go | 4 ++-- cdc/owner.go | 8 ++++---- cmd/client.go | 4 ++-- cmd/client_changefeed.go | 26 +++++++++++++------------- 5 files changed, 23 insertions(+), 23 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 6ee80ffabc3..491b8826517 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -721,7 +721,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { } //sync-point on - if c.info.SyncPoint { + if c.info.SyncPointEnabled { c.syncpointMutex.Lock() if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) @@ -839,7 +839,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { var tsUpdated bool //syncpoint on - if c.info.SyncPoint { + if c.info.SyncPointEnabled { if c.updateResolvedTs && minResolvedTs > c.status.ResolvedTs { c.status.ResolvedTs = minResolvedTs tsUpdated = true diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index fc13489a558..0874a0497cb 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -82,8 +82,8 @@ type ChangeFeedInfo struct { ErrorHis []int64 `json:"history"` Error *RunningError `json:"error"` - SyncPoint bool `json:"sync-point"` - SyncInterval time.Duration `json:"sync-interval"` + SyncPointEnabled bool `json:"sync-point-Enabled"` + SyncPointInterval time.Duration `json:"sync-point-interval"` } var changeFeedIDRe *regexp.Regexp = regexp.MustCompile(`^[a-zA-Z0-9]+(\-[a-zA-Z0-9]+)*$`) diff --git a/cdc/owner.go b/cdc/owner.go index 06f5c673aa3..2ceeaa05a02 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -534,16 +534,16 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { continue } - if newCf.info.SyncPoint { - log.Info("syncpoint is on", zap.Bool("syncpoint", newCf.info.SyncPoint)) + if newCf.info.SyncPointEnabled { + log.Info("syncpoint is on", zap.Bool("syncpoint", newCf.info.SyncPointEnabled)) //create the sync table err := newCf.createSynctable(ctx) if err != nil { return err } - newCf.startSyncPointTicker(ctx, newCf.info.SyncInterval) + newCf.startSyncPointTicker(ctx, newCf.info.SyncPointInterval) } else { - log.Info("syncpoint is off", zap.Bool("syncpoint", newCf.info.SyncPoint)) + log.Info("syncpoint is off", zap.Bool("syncpoint", newCf.info.SyncPointEnabled)) } o.changeFeeds[changeFeedID] = newCf diff --git a/cmd/client.go b/cmd/client.go index dd144b7fbf9..d30b0495c49 100644 --- a/cmd/client.go +++ b/cmd/client.go @@ -74,8 +74,8 @@ var ( captureID string interval uint - syncPoint bool - syncInterval time.Duration + syncPointEnabled bool + syncPointInterval time.Duration optForceRemove bool diff --git a/cmd/client_changefeed.go b/cmd/client_changefeed.go index 83c773584b3..8ae7b7f02d8 100644 --- a/cmd/client_changefeed.go +++ b/cmd/client_changefeed.go @@ -262,17 +262,17 @@ func verifyChangefeedParamers(ctx context.Context, cmd *cobra.Command, isCreate } } info := &model.ChangeFeedInfo{ - SinkURI: sinkURI, - Opts: make(map[string]string), - CreateTime: time.Now(), - StartTs: startTs, - TargetTs: targetTs, - Config: cfg, - Engine: model.SortEngine(sortEngine), - SortDir: sortDir, - State: model.StateNormal, - SyncPoint: syncPoint, - SyncInterval: syncInterval, + SinkURI: sinkURI, + Opts: make(map[string]string), + CreateTime: time.Now(), + StartTs: startTs, + TargetTs: targetTs, + Config: cfg, + Engine: model.SortEngine(sortEngine), + SortDir: sortDir, + State: model.StateNormal, + SyncPointEnabled: syncPointEnabled, + SyncPointInterval: syncPointInterval, } tz, err := util.GetTimezone(timezone) @@ -343,8 +343,8 @@ func changefeedConfigVariables(command *cobra.Command) { command.PersistentFlags().Uint64Var(&cyclicReplicaID, "cyclic-replica-id", 0, "(Expremental) Cyclic replication replica ID of changefeed") command.PersistentFlags().UintSliceVar(&cyclicFilterReplicaIDs, "cyclic-filter-replica-ids", []uint{}, "(Expremental) Cyclic replication filter replica ID of changefeed") command.PersistentFlags().BoolVar(&cyclicSyncDDL, "cyclic-sync-ddl", true, "(Expremental) Cyclic replication sync DDL of changefeed") - command.PersistentFlags().BoolVar(&syncPoint, "sync-point", false, "(Expremental) Set and Record syncpoint in replication(default off)") - command.PersistentFlags().DurationVar(&syncInterval, "sync-interval", 10*time.Minute, "(Expremental) Set the interval for syncpoint in replication(default 10min)") + command.PersistentFlags().BoolVar(&syncPointEnabled, "sync-point", false, "(Expremental) Set and Record syncpoint in replication(default off)") + command.PersistentFlags().DurationVar(&syncPointInterval, "sync-interval", 10*time.Minute, "(Expremental) Set the interval for syncpoint in replication(default 10min)") } func newCreateChangefeedCommand() *cobra.Command { From 09f50431c5672fc3abba3f59d395ffc4e0eb29fa Mon Sep 17 00:00:00 2001 From: colin Date: Wed, 23 Sep 2020 14:36:12 +0800 Subject: [PATCH 25/35] WrapError about mysql in changefeed --- cdc/changefeed.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 491b8826517..a057e4b0325 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -919,7 +919,7 @@ func (c *changeFeed) createSynctable(ctx context.Context) error { if err != nil { err2 := tx.Rollback() if err2 != nil { - log.Error(err2.Error()) + log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } @@ -927,7 +927,7 @@ func (c *changeFeed) createSynctable(ctx context.Context) error { if err != nil { err2 := tx.Rollback() if err2 != nil { - log.Error(err2.Error()) + log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } @@ -935,7 +935,7 @@ func (c *changeFeed) createSynctable(ctx context.Context) error { if err != nil { err2 := tx.Rollback() if err2 != nil { - log.Error(err2.Error()) + log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } @@ -957,7 +957,7 @@ func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { log.Info("sync table: get tidb_current_ts err") err2 := tx.Rollback() if err2 != nil { - log.Error(err2.Error()) + log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } @@ -965,7 +965,7 @@ func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { if err != nil { err2 := tx.Rollback() if err2 != nil { - log.Error(err2.Error()) + log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } From a15dcecc24925c68719a2b1f93437b1a77cbf456 Mon Sep 17 00:00:00 2001 From: colin Date: Wed, 30 Sep 2020 14:49:11 +0800 Subject: [PATCH 26/35] fix syncpointMutex bug in changefeed.go --- cdc/changefeed.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index a057e4b0325..33b786798bc 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -723,6 +723,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { //sync-point on if c.info.SyncPointEnabled { c.syncpointMutex.Lock() + defer c.syncpointMutex.Unlock() if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) c.updateResolvedTs = true @@ -844,7 +845,6 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { c.status.ResolvedTs = minResolvedTs tsUpdated = true } - c.syncpointMutex.Unlock() } else if minResolvedTs > c.status.ResolvedTs { c.status.ResolvedTs = minResolvedTs tsUpdated = true From 7cfa8fe83e1732c5eeeb4e00dfcb9ae96d9f2a13 Mon Sep 17 00:00:00 2001 From: colin Date: Wed, 30 Sep 2020 14:51:38 +0800 Subject: [PATCH 27/35] fix err log in changefeed.go --- cdc/changefeed.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 33b786798bc..a690bd84e7e 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -729,7 +729,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { c.updateResolvedTs = true err := c.sinkSyncpoint(ctx) if err != nil { - log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs)) + log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) } } @@ -741,7 +741,7 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) err := c.sinkSyncpoint(ctx) if err != nil { - log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs)) + log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) } c.ddlTs = 0 } From 4264afae6f21d6dc3bbb16386f9db6448735e15c Mon Sep 17 00:00:00 2001 From: colin Date: Wed, 30 Sep 2020 15:37:50 +0800 Subject: [PATCH 28/35] fix syncpoint bug with ddl(wait to exec) in changefeed.go --- cdc/changefeed.go | 60 +++++++++++++++++++++++------------------- tests/syncpoint/run.sh | 12 +++++++++ 2 files changed, 45 insertions(+), 27 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index a690bd84e7e..3358d92d3e6 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -720,33 +720,6 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { } } - //sync-point on - if c.info.SyncPointEnabled { - c.syncpointMutex.Lock() - defer c.syncpointMutex.Unlock() - if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { - log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) - c.updateResolvedTs = true - err := c.sinkSyncpoint(ctx) - if err != nil { - log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) - } - } - - if c.status.ResolvedTs == 0 { - c.updateResolvedTs = true - } - - if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs { - log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs)) - err := c.sinkSyncpoint(ctx) - if err != nil { - log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) - } - c.ddlTs = 0 - } - } - if len(c.taskPositions) < len(c.taskStatus) { log.Debug("skip update resolved ts", zap.Int("taskPositions", len(c.taskPositions)), @@ -839,6 +812,39 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { var tsUpdated bool + //sync-point on + if c.info.SyncPointEnabled { + c.syncpointMutex.Lock() + defer c.syncpointMutex.Unlock() + // ResolvedTs == CheckpointTs means a syncpoint reached; + // !c.updateResolvedTs means the syncpoint is setted by ticker; + // c.ddlTs == 0 means no DDL wait to exec and we can sink the syncpoint record securely ( c.ddlTs != 0 means some DDL should be sink to downstream and this syncpoint is fake ). + if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs && c.ddlTs == 0 { + log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) + c.updateResolvedTs = true + err := c.sinkSyncpoint(ctx) + if err != nil { + log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) + } + } + + if c.status.ResolvedTs == 0 { + c.updateResolvedTs = true + } + + // ResolvedTs == CheckpointTs means a syncpoint reached; + // ResolvedTs == ddlTs means the syncpoint is setted by DDL; + // ddlTs <= ddlExecutedTs means the DDL has been execed. + if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs && c.ddlTs <= c.ddlExecutedTs { + log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) + err := c.sinkSyncpoint(ctx) + if err != nil { + log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) + } + c.ddlTs = 0 + } + } + //syncpoint on if c.info.SyncPointEnabled { if c.updateResolvedTs && minResolvedTs > c.status.ResolvedTs { diff --git a/tests/syncpoint/run.sh b/tests/syncpoint/run.sh index 09b9ff0e134..59b4f099a79 100755 --- a/tests/syncpoint/run.sh +++ b/tests/syncpoint/run.sh @@ -26,6 +26,12 @@ function ddl() { sleep 2 run_sql "INSERT INTO testSync.simple1(id, val) VALUES (3, 3);" sleep 2 + run_sql "INSERT INTO testSync.simple1(id, val) VALUES (4, 4);" + sleep 2 + run_sql "INSERT INTO testSync.simple1(id, val) VALUES (5, 5);" + sleep 2 + run_sql "INSERT INTO testSync.simple1(id, val) VALUES (6, 6);" + sleep 2 run_sql "CREATE table testSync.simple2(id int primary key, val int);" sleep 2 run_sql "INSERT INTO testSync.simple2(id, val) VALUES (1, 1);" @@ -34,6 +40,12 @@ function ddl() { sleep 2 run_sql "INSERT INTO testSync.simple2(id, val) VALUES (3, 3);" sleep 2 + run_sql "INSERT INTO testSync.simple2(id, val) VALUES (4, 4);" + sleep 2 + run_sql "INSERT INTO testSync.simple2(id, val) VALUES (5, 5);" + sleep 2 + run_sql "INSERT INTO testSync.simple2(id, val) VALUES (6, 6);" + sleep 2 run_sql "CREATE index simple1_val ON testSync.simple1(val);" sleep 2 run_sql "CREATE index simple2_val ON testSync.simple2(val);" From 91bfda1c83162a92be4cdca9ed62ebf2025b42de Mon Sep 17 00:00:00 2001 From: colin Date: Mon, 5 Oct 2020 23:30:49 +0800 Subject: [PATCH 29/35] use raw sql.db to record syncpoint --- cdc/changefeed.go | 72 +++++++++++++++++++++++++---------------------- cdc/owner.go | 15 ++++++++++ 2 files changed, 54 insertions(+), 33 deletions(-) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 77134fdd1b8..7c94a437570 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -693,6 +693,43 @@ func (c *changeFeed) handleDDL(ctx context.Context, captures map[string]*model.C return nil } +// handleSyncPoint record every syncpoint to downstream if the syncpoint feature is enable +func (c *changeFeed) handleSyncPoint(ctx context.Context) error { + //sync-point on + if c.info.SyncPointEnabled { + c.syncpointMutex.Lock() + defer c.syncpointMutex.Unlock() + // ResolvedTs == CheckpointTs means a syncpoint reached; + // !c.updateResolvedTs means the syncpoint is setted by ticker; + // c.ddlTs == 0 means no DDL wait to exec and we can sink the syncpoint record securely ( c.ddlTs != 0 means some DDL should be sink to downstream and this syncpoint is fake ). + if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { + log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) + c.updateResolvedTs = true + err := c.sinkSyncpoint(ctx) + if err != nil { + log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) + } + } + + if c.status.ResolvedTs == 0 { + c.updateResolvedTs = true + } + + // ResolvedTs == CheckpointTs means a syncpoint reached; + // ResolvedTs == ddlTs means the syncpoint is setted by DDL; + // ddlTs <= ddlExecutedTs means the DDL has been execed. + if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs && c.ddlTs <= c.ddlExecutedTs { + log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) + err := c.sinkSyncpoint(ctx) + if err != nil { + log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) + } + c.ddlTs = 0 + } + } + return nil +} + // calcResolvedTs update every changefeed's resolve ts and checkpoint ts. func (c *changeFeed) calcResolvedTs(ctx context.Context) error { if c.ddlState != model.ChangeFeedSyncDML && c.ddlState != model.ChangeFeedWaitToExecDDL { @@ -815,45 +852,14 @@ func (c *changeFeed) calcResolvedTs(ctx context.Context) error { var tsUpdated bool - //sync-point on - if c.info.SyncPointEnabled { - c.syncpointMutex.Lock() - defer c.syncpointMutex.Unlock() - // ResolvedTs == CheckpointTs means a syncpoint reached; - // !c.updateResolvedTs means the syncpoint is setted by ticker; - // c.ddlTs == 0 means no DDL wait to exec and we can sink the syncpoint record securely ( c.ddlTs != 0 means some DDL should be sink to downstream and this syncpoint is fake ). - if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs && c.ddlTs == 0 { - log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) - c.updateResolvedTs = true - err := c.sinkSyncpoint(ctx) - if err != nil { - log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) - } - } - - if c.status.ResolvedTs == 0 { - c.updateResolvedTs = true - } - - // ResolvedTs == CheckpointTs means a syncpoint reached; - // ResolvedTs == ddlTs means the syncpoint is setted by DDL; - // ddlTs <= ddlExecutedTs means the DDL has been execed. - if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs && c.ddlTs <= c.ddlExecutedTs { - log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) - err := c.sinkSyncpoint(ctx) - if err != nil { - log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) - } - c.ddlTs = 0 - } - } - //syncpoint on if c.info.SyncPointEnabled { + c.syncpointMutex.Lock() if c.updateResolvedTs && minResolvedTs > c.status.ResolvedTs { c.status.ResolvedTs = minResolvedTs tsUpdated = true } + c.syncpointMutex.Unlock() } else if minResolvedTs > c.status.ResolvedTs { c.status.ResolvedTs = minResolvedTs tsUpdated = true diff --git a/cdc/owner.go b/cdc/owner.go index 2ceeaa05a02..57f38ee7923 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -670,6 +670,16 @@ func (o *Owner) handleDDL(ctx context.Context) error { return nil } +// handleSyncPoint call handleSyncPoint of every changefeeds +func (o *Owner) handleSyncPoint(ctx context.Context) error { + for _, cf := range o.changeFeeds { + if err := cf.handleSyncPoint(ctx); err != nil { + return errors.Trace(err) + } + } + return nil +} + // dispatchJob dispatches job to processors func (o *Owner) dispatchJob(ctx context.Context, job model.AdminJob) error { cf, ok := o.changeFeeds[job.CfID] @@ -1054,6 +1064,11 @@ func (o *Owner) run(ctx context.Context) error { return errors.Trace(err) } + err = o.handleSyncPoint(ctx) + if err != nil { + return errors.Trace(err) + } + err = o.handleAdminJob(ctx) if err != nil { return errors.Trace(err) From 3c846af6de060793c04befd7f55e7e37d33f6620 Mon Sep 17 00:00:00 2001 From: colin Date: Tue, 6 Oct 2020 00:54:14 +0800 Subject: [PATCH 30/35] Diversified ts-map storage implementation --- cdc/changefeed.go | 79 ++++------------------------------- cdc/owner.go | 14 ++++--- cdc/sink/mysql.go | 87 +++++++++++++++++++++++++++++++++++---- cdc/sink/syncpointSink.go | 50 ++++++++++++++++++++++ 4 files changed, 147 insertions(+), 83 deletions(-) create mode 100644 cdc/sink/syncpointSink.go diff --git a/cdc/changefeed.go b/cdc/changefeed.go index 7c94a437570..f2b0523808f 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -15,7 +15,6 @@ package cdc import ( "context" - "database/sql" "fmt" "math" "sync" @@ -87,7 +86,7 @@ type changeFeed struct { syncpointMutex sync.Mutex updateResolvedTs bool startTimer chan bool - syncDB *sql.DB + syncpointSink sink.SyncpointSink syncCancel context.CancelFunc taskStatus model.ProcessorsInfos taskPositions map[model.CaptureID]*model.TaskPosition @@ -705,7 +704,7 @@ func (c *changeFeed) handleSyncPoint(ctx context.Context) error { if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) c.updateResolvedTs = true - err := c.sinkSyncpoint(ctx) + err := c.syncpointSink.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) if err != nil { log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) } @@ -720,7 +719,7 @@ func (c *changeFeed) handleSyncPoint(ctx context.Context) error { // ddlTs <= ddlExecutedTs means the DDL has been execed. if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs && c.ddlTs <= c.ddlExecutedTs { log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) - err := c.sinkSyncpoint(ctx) + err := c.syncpointSink.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) if err != nil { log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) } @@ -922,72 +921,6 @@ func (c *changeFeed) startSyncPeriod(ctx context.Context, interval time.Duration }(ctx) } -//createSynctable create a sync table to record the -func (c *changeFeed) createSynctable(ctx context.Context) error { - database := "TiCDC" - tx, err := c.syncDB.BeginTx(ctx, nil) - if err != nil { - log.Info("create sync table: begin Tx fail") - return cerror.WrapError(cerror.ErrMySQLTxnError, err) - } - _, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database) - if err != nil { - err2 := tx.Rollback() - if err2 != nil { - log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) - } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) - } - _, err = tx.Exec("USE " + database) - if err != nil { - err2 := tx.Rollback() - if err2 != nil { - log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) - } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) - } - _, err = tx.Exec("CREATE TABLE IF NOT EXISTS syncpoint (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `cf`, `primary_ts` ) )") - if err != nil { - err2 := tx.Rollback() - if err2 != nil { - log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) - } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) - } - err = tx.Commit() - return cerror.WrapError(cerror.ErrMySQLTxnError, err) -} - -//sinkSyncpoint record the syncpoint(a map with ts) in downstream db -func (c *changeFeed) sinkSyncpoint(ctx context.Context) error { - tx, err := c.syncDB.BeginTx(ctx, nil) - if err != nil { - log.Info("sync table: begin Tx fail") - return cerror.WrapError(cerror.ErrMySQLTxnError, err) - } - row := tx.QueryRow("select @@tidb_current_ts") - var secondaryTs string - err = row.Scan(&secondaryTs) - if err != nil { - log.Info("sync table: get tidb_current_ts err") - err2 := tx.Rollback() - if err2 != nil { - log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) - } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) - } - _, err = tx.Exec("insert into TiCDC.syncpoint(cf, primary_ts, secondary_ts) VALUES (?,?,?)", c.id, c.status.CheckpointTs, secondaryTs) - if err != nil { - err2 := tx.Rollback() - if err2 != nil { - log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) - } - return cerror.WrapError(cerror.ErrMySQLTxnError, err) - } - err = tx.Commit() - return cerror.WrapError(cerror.ErrMySQLTxnError, err) -} - func (c *changeFeed) stopSyncPointTicker() { if c.syncCancel != nil { c.syncCancel() @@ -1010,5 +943,11 @@ func (c *changeFeed) Close() { if err != nil { log.Warn("failed to close owner sink", zap.Error(err)) } + if c.syncpointSink != nil { + err = c.syncpointSink.Close() + if err != nil { + log.Warn("failed to close owner sink", zap.Error(err)) + } + } log.Info("changefeed closed", zap.String("id", c.id)) } diff --git a/cdc/owner.go b/cdc/owner.go index 57f38ee7923..a3e26f8af95 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -357,9 +357,13 @@ func (o *Owner) newChangeFeed( log.Error("error on running owner", zap.Error(err)) } - syncDB, err := sink.NewSyncpointSinklink(ctx, info, id) - if err != nil { - return nil, errors.Trace(err) + var syncpointSink sink.SyncpointSink + syncpointSink = nil + if info.SyncPointEnabled { + syncpointSink, err = sink.NewSyncpointSink(ctx, id, info.SinkURI) + if err != nil { + return nil, errors.Trace(err) + } } cf = &changeFeed{ @@ -383,7 +387,7 @@ func (o *Owner) newChangeFeed( ddlTs: 0, updateResolvedTs: true, startTimer: make(chan bool), - syncDB: syncDB, + syncpointSink: syncpointSink, syncCancel: nil, taskStatus: processorsInfos, taskPositions: taskPositions, @@ -537,7 +541,7 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { if newCf.info.SyncPointEnabled { log.Info("syncpoint is on", zap.Bool("syncpoint", newCf.info.SyncPointEnabled)) //create the sync table - err := newCf.createSynctable(ctx) + err := newCf.syncpointSink.CreateSynctable(ctx) if err != nil { return err } diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 82e602b93c7..26fab866674 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -74,6 +74,9 @@ var ( } ) +type mysqlSyncpointSink struct { + db *sql.DB +} type mysqlSink struct { db *sql.DB params *sinkParams @@ -1102,14 +1105,10 @@ func buildColumnList(names []string) string { return b.String() } -//NewSyncpointSinklink create a sink to record the syncpoint map in downstream DB for every changefeed -func NewSyncpointSinklink(ctx context.Context, info *model.ChangeFeedInfo, id string) (*sql.DB, error) { +// newSyncpointSink create a sink to record the syncpoint map in downstream DB for every changefeed +func newMySQLSyncpointSink(ctx context.Context, id string, sinkURI *url.URL) (SyncpointSink, error) { var syncDB *sql.DB - // parse sinkURI as a URI - sinkURI, err := url.Parse(info.SinkURI) - if err != nil { - return nil, errors.Annotatef(err, "parse sinkURI failed") - } + //todo If is neither mysql nor tidb, such as kafka, just ignore this feature. scheme := strings.ToLower(sinkURI.Scheme) if scheme != "mysql" && scheme != "tidb" && scheme != "mysql+ssl" && scheme != "tidb+ssl" { @@ -1187,6 +1186,78 @@ func NewSyncpointSinklink(ctx context.Context, info *model.ChangeFeedInfo, id st } log.Info("Start mysql syncpoint sink") + syncpointSink := &mysqlSyncpointSink{ + db: syncDB, + } - return syncDB, nil + return syncpointSink, nil +} + +func (s *mysqlSyncpointSink) CreateSynctable(ctx context.Context) error { + database := "TiCDC" + tx, err := s.db.BeginTx(ctx, nil) + if err != nil { + log.Info("create sync table: begin Tx fail") + return cerror.WrapError(cerror.ErrMySQLTxnError, err) + } + _, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database) + if err != nil { + err2 := tx.Rollback() + if err2 != nil { + log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) + } + return cerror.WrapError(cerror.ErrMySQLTxnError, err) + } + _, err = tx.Exec("USE " + database) + if err != nil { + err2 := tx.Rollback() + if err2 != nil { + log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) + } + return cerror.WrapError(cerror.ErrMySQLTxnError, err) + } + _, err = tx.Exec("CREATE TABLE IF NOT EXISTS syncpoint (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `cf`, `primary_ts` ) )") + if err != nil { + err2 := tx.Rollback() + if err2 != nil { + log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) + } + return cerror.WrapError(cerror.ErrMySQLTxnError, err) + } + err = tx.Commit() + return cerror.WrapError(cerror.ErrMySQLTxnError, err) +} + +func (s *mysqlSyncpointSink) SinkSyncpoint(ctx context.Context, id string, checkpointTs uint64) error { + tx, err := s.db.BeginTx(ctx, nil) + if err != nil { + log.Info("sync table: begin Tx fail") + return cerror.WrapError(cerror.ErrMySQLTxnError, err) + } + row := tx.QueryRow("select @@tidb_current_ts") + var secondaryTs string + err = row.Scan(&secondaryTs) + if err != nil { + log.Info("sync table: get tidb_current_ts err") + err2 := tx.Rollback() + if err2 != nil { + log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) + } + return cerror.WrapError(cerror.ErrMySQLTxnError, err) + } + _, err = tx.Exec("insert into TiCDC.syncpoint(cf, primary_ts, secondary_ts) VALUES (?,?,?)", id, checkpointTs, secondaryTs) + if err != nil { + err2 := tx.Rollback() + if err2 != nil { + log.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2).Error()) + } + return cerror.WrapError(cerror.ErrMySQLTxnError, err) + } + err = tx.Commit() + return cerror.WrapError(cerror.ErrMySQLTxnError, err) +} + +func (s *mysqlSyncpointSink) Close() error { + err := s.db.Close() + return cerror.WrapError(cerror.ErrMySQLConnectionError, err) } diff --git a/cdc/sink/syncpointSink.go b/cdc/sink/syncpointSink.go new file mode 100644 index 00000000000..83f12278a08 --- /dev/null +++ b/cdc/sink/syncpointSink.go @@ -0,0 +1,50 @@ +// Copyright 2020 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 sink + +import ( + "context" + "net/url" + "strings" + + "github.com/pingcap/ticdc/cdc/model" + cerror "github.com/pingcap/ticdc/pkg/errors" +) + +// SyncpointSink is an abstraction for anything that a changefeed may emit into. +type SyncpointSink interface { + // CreateSynctable create a table to record the syncpoints + CreateSynctable(ctx context.Context) error + + // SinkSyncpoint record the syncpoint(a map with ts) in downstream db + SinkSyncpoint(ctx context.Context, id string, checkpointTs uint64) error + + // Close closes the SyncpointSink + Close() error +} + +// NewSyncpointSink creates a new Spyncpoint sink with the sink-uri +func NewSyncpointSink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURIStr string) (SyncpointSink, error) { + // parse sinkURI as a URI + sinkURI, err := url.Parse(sinkURIStr) + if err != nil { + return nil, cerror.WrapError(cerror.ErrSinkURIInvalid, err) + } + switch strings.ToLower(sinkURI.Scheme) { + case "mysql", "tidb", "mysql+ssl", "tidb+ssl": + return newMySQLSyncpointSink(ctx, changefeedID, sinkURI) + default: + return nil, cerror.ErrSinkURIInvalid.GenWithStack("the sink scheme (%s) is not supported", sinkURI.Scheme) + } +} From 475319961d5ef2af1d50edf88b0bdd6b26999161 Mon Sep 17 00:00:00 2001 From: colin Date: Fri, 9 Oct 2020 14:50:22 +0800 Subject: [PATCH 31/35] fix a liitle code about format --- cdc/model/changefeed.go | 2 +- cdc/owner.go | 1 - cdc/sink/mysql.go | 6 +++--- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 0874a0497cb..2043ce66c5a 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -82,7 +82,7 @@ type ChangeFeedInfo struct { ErrorHis []int64 `json:"history"` Error *RunningError `json:"error"` - SyncPointEnabled bool `json:"sync-point-Enabled"` + SyncPointEnabled bool `json:"sync-point-enabled"` SyncPointInterval time.Duration `json:"sync-point-interval"` } diff --git a/cdc/owner.go b/cdc/owner.go index a3e26f8af95..413a5e0c354 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -358,7 +358,6 @@ func (o *Owner) newChangeFeed( } var syncpointSink sink.SyncpointSink - syncpointSink = nil if info.SyncPointEnabled { syncpointSink, err = sink.NewSyncpointSink(ctx, id, info.SinkURI) if err != nil { diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 26fab866674..69baac57249 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -1182,7 +1182,7 @@ func newMySQLSyncpointSink(ctx context.Context, id string, sinkURI *url.URL) (Sy } err = syncDB.PingContext(ctx) if err != nil { - return nil, errors.Annotatef(err, "fail to open MySQL connection") + return nil, errors.Annotate(err, "fail to open MySQL connection") } log.Info("Start mysql syncpoint sink") @@ -1197,7 +1197,7 @@ func (s *mysqlSyncpointSink) CreateSynctable(ctx context.Context) error { database := "TiCDC" tx, err := s.db.BeginTx(ctx, nil) if err != nil { - log.Info("create sync table: begin Tx fail") + log.Error("create sync table: begin Tx fail", zap.Error(err)) return cerror.WrapError(cerror.ErrMySQLTxnError, err) } _, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database) @@ -1231,7 +1231,7 @@ func (s *mysqlSyncpointSink) CreateSynctable(ctx context.Context) error { func (s *mysqlSyncpointSink) SinkSyncpoint(ctx context.Context, id string, checkpointTs uint64) error { tx, err := s.db.BeginTx(ctx, nil) if err != nil { - log.Info("sync table: begin Tx fail") + log.Error("sync table: begin Tx fail", zap.Error(err)) return cerror.WrapError(cerror.ErrMySQLTxnError, err) } row := tx.QueryRow("select @@tidb_current_ts") From f6f3993d29b1f9f7548ce801b2817bd3fc3b6fc6 Mon Sep 17 00:00:00 2001 From: colin Date: Fri, 9 Oct 2020 21:59:40 +0800 Subject: [PATCH 32/35] change some info in log(owner.go) --- cdc/owner.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cdc/owner.go b/cdc/owner.go index 413a5e0c354..d0bb7e71eff 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -538,7 +538,7 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { } if newCf.info.SyncPointEnabled { - log.Info("syncpoint is on", zap.Bool("syncpoint", newCf.info.SyncPointEnabled)) + log.Info("syncpoint is on, creating the sync table") //create the sync table err := newCf.syncpointSink.CreateSynctable(ctx) if err != nil { @@ -546,7 +546,7 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { } newCf.startSyncPointTicker(ctx, newCf.info.SyncPointInterval) } else { - log.Info("syncpoint is off", zap.Bool("syncpoint", newCf.info.SyncPointEnabled)) + log.Info("syncpoint is off") } o.changeFeeds[changeFeedID] = newCf From 6ab63c7818d4bcdd307da704ce4805fa43e14553 Mon Sep 17 00:00:00 2001 From: colin Date: Sat, 10 Oct 2020 16:23:11 +0800 Subject: [PATCH 33/35] set syncpoint's tablename and schema name to be constants --- cdc/sink/mysql.go | 9 ++++++--- tests/syncpoint/run.sh | 2 +- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 69baac57249..1201097ef2b 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -65,6 +65,9 @@ const ( defaultSafeMode = true ) +// SyncpointTableName is the name of table where all syncpoint maps sit +const syncpointTableName string = "syncpoint_v1" + var ( validSchemes = map[string]bool{ "mysql": true, @@ -1194,7 +1197,7 @@ func newMySQLSyncpointSink(ctx context.Context, id string, sinkURI *url.URL) (Sy } func (s *mysqlSyncpointSink) CreateSynctable(ctx context.Context) error { - database := "TiCDC" + database := mark.SchemaName tx, err := s.db.BeginTx(ctx, nil) if err != nil { log.Error("create sync table: begin Tx fail", zap.Error(err)) @@ -1216,7 +1219,7 @@ func (s *mysqlSyncpointSink) CreateSynctable(ctx context.Context) error { } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } - _, err = tx.Exec("CREATE TABLE IF NOT EXISTS syncpoint (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `cf`, `primary_ts` ) )") + _, err = tx.Exec("CREATE TABLE IF NOT EXISTS " + syncpointTableName + " (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `cf`, `primary_ts` ) )") if err != nil { err2 := tx.Rollback() if err2 != nil { @@ -1245,7 +1248,7 @@ func (s *mysqlSyncpointSink) SinkSyncpoint(ctx context.Context, id string, check } return cerror.WrapError(cerror.ErrMySQLTxnError, err) } - _, err = tx.Exec("insert into TiCDC.syncpoint(cf, primary_ts, secondary_ts) VALUES (?,?,?)", id, checkpointTs, secondaryTs) + _, err = tx.Exec("insert into "+mark.SchemaName+"."+syncpointTableName+"(cf, primary_ts, secondary_ts) VALUES (?,?,?)", id, checkpointTs, secondaryTs) if err != nil { err2 := tx.Rollback() if err2 != nil { diff --git a/tests/syncpoint/run.sh b/tests/syncpoint/run.sh index 59b4f099a79..d75466fb311 100755 --- a/tests/syncpoint/run.sh +++ b/tests/syncpoint/run.sh @@ -121,7 +121,7 @@ function run() { sleep 60 - run_sql "SELECT primary_ts, secondary_ts FROM TiCDC.syncpoint;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "SELECT primary_ts, secondary_ts FROM tidb_cdc.syncpoint_v1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} echo "____________________________________" cat "$OUT_DIR/sql_res.$TEST_NAME.txt" checkDiff From 416baee03c2e8d919d5e3d6af722dfd26fd6be55 Mon Sep 17 00:00:00 2001 From: colin Date: Sat, 10 Oct 2020 16:51:50 +0800 Subject: [PATCH 34/35] change name 'syncpointSink' to 'syncpointLink' --- cdc/changefeed.go | 10 +++++----- cdc/owner.go | 8 ++++---- cdc/sink/mysql.go | 16 ++++++++-------- cdc/sink/{syncpointSink.go => syncpointLink.go} | 10 +++++----- 4 files changed, 22 insertions(+), 22 deletions(-) rename cdc/sink/{syncpointSink.go => syncpointLink.go} (79%) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index f2b0523808f..cf95bd19f4c 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -86,7 +86,7 @@ type changeFeed struct { syncpointMutex sync.Mutex updateResolvedTs bool startTimer chan bool - syncpointSink sink.SyncpointSink + syncpointLink sink.SyncpointLink syncCancel context.CancelFunc taskStatus model.ProcessorsInfos taskPositions map[model.CaptureID]*model.TaskPosition @@ -704,7 +704,7 @@ func (c *changeFeed) handleSyncPoint(ctx context.Context) error { if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) c.updateResolvedTs = true - err := c.syncpointSink.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) + err := c.syncpointLink.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) if err != nil { log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) } @@ -719,7 +719,7 @@ func (c *changeFeed) handleSyncPoint(ctx context.Context) error { // ddlTs <= ddlExecutedTs means the DDL has been execed. if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs && c.ddlTs <= c.ddlExecutedTs { log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) - err := c.syncpointSink.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) + err := c.syncpointLink.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) if err != nil { log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) } @@ -943,8 +943,8 @@ func (c *changeFeed) Close() { if err != nil { log.Warn("failed to close owner sink", zap.Error(err)) } - if c.syncpointSink != nil { - err = c.syncpointSink.Close() + if c.syncpointLink != nil { + err = c.syncpointLink.Close() if err != nil { log.Warn("failed to close owner sink", zap.Error(err)) } diff --git a/cdc/owner.go b/cdc/owner.go index d0bb7e71eff..506267f663a 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -357,9 +357,9 @@ func (o *Owner) newChangeFeed( log.Error("error on running owner", zap.Error(err)) } - var syncpointSink sink.SyncpointSink + var syncpointLink sink.SyncpointLink if info.SyncPointEnabled { - syncpointSink, err = sink.NewSyncpointSink(ctx, id, info.SinkURI) + syncpointLink, err = sink.NewSyncpointLink(ctx, id, info.SinkURI) if err != nil { return nil, errors.Trace(err) } @@ -386,7 +386,7 @@ func (o *Owner) newChangeFeed( ddlTs: 0, updateResolvedTs: true, startTimer: make(chan bool), - syncpointSink: syncpointSink, + syncpointLink: syncpointLink, syncCancel: nil, taskStatus: processorsInfos, taskPositions: taskPositions, @@ -540,7 +540,7 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { if newCf.info.SyncPointEnabled { log.Info("syncpoint is on, creating the sync table") //create the sync table - err := newCf.syncpointSink.CreateSynctable(ctx) + err := newCf.syncpointLink.CreateSynctable(ctx) if err != nil { return err } diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 1201097ef2b..f3395b4af39 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -77,7 +77,7 @@ var ( } ) -type mysqlSyncpointSink struct { +type mysqlSyncpointLink struct { db *sql.DB } type mysqlSink struct { @@ -1108,8 +1108,8 @@ func buildColumnList(names []string) string { return b.String() } -// newSyncpointSink create a sink to record the syncpoint map in downstream DB for every changefeed -func newMySQLSyncpointSink(ctx context.Context, id string, sinkURI *url.URL) (SyncpointSink, error) { +// newSyncpointLink create a sink to record the syncpoint map in downstream DB for every changefeed +func newMySQLSyncpointLink(ctx context.Context, id string, sinkURI *url.URL) (SyncpointLink, error) { var syncDB *sql.DB //todo If is neither mysql nor tidb, such as kafka, just ignore this feature. @@ -1189,14 +1189,14 @@ func newMySQLSyncpointSink(ctx context.Context, id string, sinkURI *url.URL) (Sy } log.Info("Start mysql syncpoint sink") - syncpointSink := &mysqlSyncpointSink{ + syncpointLink := &mysqlSyncpointLink{ db: syncDB, } - return syncpointSink, nil + return syncpointLink, nil } -func (s *mysqlSyncpointSink) CreateSynctable(ctx context.Context) error { +func (s *mysqlSyncpointLink) CreateSynctable(ctx context.Context) error { database := mark.SchemaName tx, err := s.db.BeginTx(ctx, nil) if err != nil { @@ -1231,7 +1231,7 @@ func (s *mysqlSyncpointSink) CreateSynctable(ctx context.Context) error { return cerror.WrapError(cerror.ErrMySQLTxnError, err) } -func (s *mysqlSyncpointSink) SinkSyncpoint(ctx context.Context, id string, checkpointTs uint64) error { +func (s *mysqlSyncpointLink) SinkSyncpoint(ctx context.Context, id string, checkpointTs uint64) error { tx, err := s.db.BeginTx(ctx, nil) if err != nil { log.Error("sync table: begin Tx fail", zap.Error(err)) @@ -1260,7 +1260,7 @@ func (s *mysqlSyncpointSink) SinkSyncpoint(ctx context.Context, id string, check return cerror.WrapError(cerror.ErrMySQLTxnError, err) } -func (s *mysqlSyncpointSink) Close() error { +func (s *mysqlSyncpointLink) Close() error { err := s.db.Close() return cerror.WrapError(cerror.ErrMySQLConnectionError, err) } diff --git a/cdc/sink/syncpointSink.go b/cdc/sink/syncpointLink.go similarity index 79% rename from cdc/sink/syncpointSink.go rename to cdc/sink/syncpointLink.go index 83f12278a08..905442d2954 100644 --- a/cdc/sink/syncpointSink.go +++ b/cdc/sink/syncpointLink.go @@ -22,8 +22,8 @@ import ( cerror "github.com/pingcap/ticdc/pkg/errors" ) -// SyncpointSink is an abstraction for anything that a changefeed may emit into. -type SyncpointSink interface { +// SyncpointLink is an abstraction for anything that a changefeed may emit into. +type SyncpointLink interface { // CreateSynctable create a table to record the syncpoints CreateSynctable(ctx context.Context) error @@ -34,8 +34,8 @@ type SyncpointSink interface { Close() error } -// NewSyncpointSink creates a new Spyncpoint sink with the sink-uri -func NewSyncpointSink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURIStr string) (SyncpointSink, error) { +// NewSyncpointLink creates a new Spyncpoint sink with the sink-uri +func NewSyncpointLink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURIStr string) (SyncpointLink, error) { // parse sinkURI as a URI sinkURI, err := url.Parse(sinkURIStr) if err != nil { @@ -43,7 +43,7 @@ func NewSyncpointSink(ctx context.Context, changefeedID model.ChangeFeedID, sink } switch strings.ToLower(sinkURI.Scheme) { case "mysql", "tidb", "mysql+ssl", "tidb+ssl": - return newMySQLSyncpointSink(ctx, changefeedID, sinkURI) + return newMySQLSyncpointLink(ctx, changefeedID, sinkURI) default: return nil, cerror.ErrSinkURIInvalid.GenWithStack("the sink scheme (%s) is not supported", sinkURI.Scheme) } From 5045d090c3aca88167ec0715fc4708248ae18149 Mon Sep 17 00:00:00 2001 From: colin Date: Sat, 10 Oct 2020 19:05:09 +0800 Subject: [PATCH 35/35] change name 'syncpointLink' to 'syncpointStore' --- cdc/changefeed.go | 10 +++++----- cdc/owner.go | 8 ++++---- cdc/sink/mysql.go | 16 ++++++++-------- cdc/sink/{syncpointLink.go => syncpointStore.go} | 10 +++++----- 4 files changed, 22 insertions(+), 22 deletions(-) rename cdc/sink/{syncpointLink.go => syncpointStore.go} (78%) diff --git a/cdc/changefeed.go b/cdc/changefeed.go index cf95bd19f4c..750928fa6d0 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -86,7 +86,7 @@ type changeFeed struct { syncpointMutex sync.Mutex updateResolvedTs bool startTimer chan bool - syncpointLink sink.SyncpointLink + syncpointStore sink.SyncpointStore syncCancel context.CancelFunc taskStatus model.ProcessorsInfos taskPositions map[model.CaptureID]*model.TaskPosition @@ -704,7 +704,7 @@ func (c *changeFeed) handleSyncPoint(ctx context.Context) error { if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) c.updateResolvedTs = true - err := c.syncpointLink.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) + err := c.syncpointStore.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) if err != nil { log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) } @@ -719,7 +719,7 @@ func (c *changeFeed) handleSyncPoint(ctx context.Context) error { // ddlTs <= ddlExecutedTs means the DDL has been execed. if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs && c.ddlTs <= c.ddlExecutedTs { log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) - err := c.syncpointLink.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) + err := c.syncpointStore.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) if err != nil { log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) } @@ -943,8 +943,8 @@ func (c *changeFeed) Close() { if err != nil { log.Warn("failed to close owner sink", zap.Error(err)) } - if c.syncpointLink != nil { - err = c.syncpointLink.Close() + if c.syncpointStore != nil { + err = c.syncpointStore.Close() if err != nil { log.Warn("failed to close owner sink", zap.Error(err)) } diff --git a/cdc/owner.go b/cdc/owner.go index 506267f663a..860828185df 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -357,9 +357,9 @@ func (o *Owner) newChangeFeed( log.Error("error on running owner", zap.Error(err)) } - var syncpointLink sink.SyncpointLink + var syncpointStore sink.SyncpointStore if info.SyncPointEnabled { - syncpointLink, err = sink.NewSyncpointLink(ctx, id, info.SinkURI) + syncpointStore, err = sink.NewSyncpointStore(ctx, id, info.SinkURI) if err != nil { return nil, errors.Trace(err) } @@ -386,7 +386,7 @@ func (o *Owner) newChangeFeed( ddlTs: 0, updateResolvedTs: true, startTimer: make(chan bool), - syncpointLink: syncpointLink, + syncpointStore: syncpointStore, syncCancel: nil, taskStatus: processorsInfos, taskPositions: taskPositions, @@ -540,7 +540,7 @@ func (o *Owner) loadChangeFeeds(ctx context.Context) error { if newCf.info.SyncPointEnabled { log.Info("syncpoint is on, creating the sync table") //create the sync table - err := newCf.syncpointLink.CreateSynctable(ctx) + err := newCf.syncpointStore.CreateSynctable(ctx) if err != nil { return err } diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index f3395b4af39..a76dc940706 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -77,7 +77,7 @@ var ( } ) -type mysqlSyncpointLink struct { +type mysqlSyncpointStore struct { db *sql.DB } type mysqlSink struct { @@ -1108,8 +1108,8 @@ func buildColumnList(names []string) string { return b.String() } -// newSyncpointLink create a sink to record the syncpoint map in downstream DB for every changefeed -func newMySQLSyncpointLink(ctx context.Context, id string, sinkURI *url.URL) (SyncpointLink, error) { +// newSyncpointStore create a sink to record the syncpoint map in downstream DB for every changefeed +func newMySQLSyncpointStore(ctx context.Context, id string, sinkURI *url.URL) (SyncpointStore, error) { var syncDB *sql.DB //todo If is neither mysql nor tidb, such as kafka, just ignore this feature. @@ -1189,14 +1189,14 @@ func newMySQLSyncpointLink(ctx context.Context, id string, sinkURI *url.URL) (Sy } log.Info("Start mysql syncpoint sink") - syncpointLink := &mysqlSyncpointLink{ + syncpointStore := &mysqlSyncpointStore{ db: syncDB, } - return syncpointLink, nil + return syncpointStore, nil } -func (s *mysqlSyncpointLink) CreateSynctable(ctx context.Context) error { +func (s *mysqlSyncpointStore) CreateSynctable(ctx context.Context) error { database := mark.SchemaName tx, err := s.db.BeginTx(ctx, nil) if err != nil { @@ -1231,7 +1231,7 @@ func (s *mysqlSyncpointLink) CreateSynctable(ctx context.Context) error { return cerror.WrapError(cerror.ErrMySQLTxnError, err) } -func (s *mysqlSyncpointLink) SinkSyncpoint(ctx context.Context, id string, checkpointTs uint64) error { +func (s *mysqlSyncpointStore) SinkSyncpoint(ctx context.Context, id string, checkpointTs uint64) error { tx, err := s.db.BeginTx(ctx, nil) if err != nil { log.Error("sync table: begin Tx fail", zap.Error(err)) @@ -1260,7 +1260,7 @@ func (s *mysqlSyncpointLink) SinkSyncpoint(ctx context.Context, id string, check return cerror.WrapError(cerror.ErrMySQLTxnError, err) } -func (s *mysqlSyncpointLink) Close() error { +func (s *mysqlSyncpointStore) Close() error { err := s.db.Close() return cerror.WrapError(cerror.ErrMySQLConnectionError, err) } diff --git a/cdc/sink/syncpointLink.go b/cdc/sink/syncpointStore.go similarity index 78% rename from cdc/sink/syncpointLink.go rename to cdc/sink/syncpointStore.go index 905442d2954..d20cb5e5960 100644 --- a/cdc/sink/syncpointLink.go +++ b/cdc/sink/syncpointStore.go @@ -22,8 +22,8 @@ import ( cerror "github.com/pingcap/ticdc/pkg/errors" ) -// SyncpointLink is an abstraction for anything that a changefeed may emit into. -type SyncpointLink interface { +// SyncpointStore is an abstraction for anything that a changefeed may emit into. +type SyncpointStore interface { // CreateSynctable create a table to record the syncpoints CreateSynctable(ctx context.Context) error @@ -34,8 +34,8 @@ type SyncpointLink interface { Close() error } -// NewSyncpointLink creates a new Spyncpoint sink with the sink-uri -func NewSyncpointLink(ctx context.Context, changefeedID model.ChangeFeedID, sinkURIStr string) (SyncpointLink, error) { +// NewSyncpointStore creates a new Spyncpoint sink with the sink-uri +func NewSyncpointStore(ctx context.Context, changefeedID model.ChangeFeedID, sinkURIStr string) (SyncpointStore, error) { // parse sinkURI as a URI sinkURI, err := url.Parse(sinkURIStr) if err != nil { @@ -43,7 +43,7 @@ func NewSyncpointLink(ctx context.Context, changefeedID model.ChangeFeedID, sink } switch strings.ToLower(sinkURI.Scheme) { case "mysql", "tidb", "mysql+ssl", "tidb+ssl": - return newMySQLSyncpointLink(ctx, changefeedID, sinkURI) + return newMySQLSyncpointStore(ctx, changefeedID, sinkURI) default: return nil, cerror.ErrSinkURIInvalid.GenWithStack("the sink scheme (%s) is not supported", sinkURI.Scheme) }