From 0f06a11a71d475de436e42fbdea06d6cc553e525 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 7 Aug 2024 19:44:56 +0800 Subject: [PATCH 01/50] sync to office --- cdc/owner/ddl_sink.go | 1 + cdc/sink/dmlsink/txn/mysql/mysql.go | 3 + cdc/sink/validator/validator.go | 2 +- pkg/sink/mysql/db_helper.go | 158 +++++++++++++++++---------- tests/integration_tests/run_group.sh | 2 +- 5 files changed, 104 insertions(+), 62 deletions(-) diff --git a/cdc/owner/ddl_sink.go b/cdc/owner/ddl_sink.go index 4de195ac0b5..0f842ece6f1 100644 --- a/cdc/owner/ddl_sink.go +++ b/cdc/owner/ddl_sink.go @@ -391,6 +391,7 @@ func (s *ddlSinkImpl) emitSyncPoint(ctx context.Context, checkpointTs uint64) (e for { if err = s.makeSyncPointStoreReady(ctx); err == nil { // TODO implement async sink syncPoint + // TODO: err = s.syncPointStore.SinkSyncPoint(ctx, s.changefeedID, checkpointTs) } if err == nil { diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index dea8debe570..39ee3578e7d 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -97,6 +97,7 @@ func NewMySQLBackends( return nil, err } + // TODO: 这里生成多个 dsnStr 返回 dsnStr, err := pmysql.GenerateDSN(ctx, sinkURI, cfg, dbConnFactory) if err != nil { return nil, err @@ -766,6 +767,8 @@ func (s *mysqlBackend) execDMLWithMaxRetries(pctx context.Context, dmls *prepare return 0, 0, err } + // mysql 8.0 5.7 + // If interplated SQL size exceeds maxAllowedPacket, mysql driver will // fall back to the sequantial way. // error can be ErrPrepareMulti, ErrBadConn etc. diff --git a/cdc/sink/validator/validator.go b/cdc/sink/validator/validator.go index 161002e3381..ef3b0842464 100644 --- a/cdc/sink/validator/validator.go +++ b/cdc/sink/validator/validator.go @@ -122,7 +122,7 @@ func checkBDRMode(ctx context.Context, sinkURI *url.URL, replicaConfig *config.R if err != nil { return err } - dsn, err := pmysql.GenBasicDSN(sinkURI, cfg) + dsn, err := pmysql.GetDesinationCfg(sinkURI, cfg) if err != nil { return err } diff --git a/pkg/sink/mysql/db_helper.go b/pkg/sink/mysql/db_helper.go index 1a3d3c8e84c..b351d6952ff 100644 --- a/pkg/sink/mysql/db_helper.go +++ b/pkg/sink/mysql/db_helper.go @@ -18,23 +18,41 @@ import ( "database/sql" "encoding/base64" "fmt" - "net" "net/url" "strconv" + "strings" dmysql "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tidb/pkg/parser/charset" - "github.com/pingcap/tidb/pkg/parser/mysql" tmysql "github.com/pingcap/tidb/pkg/parser/mysql" dmutils "github.com/pingcap/tiflow/dm/pkg/conn" cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) +func ConnectOneAvailableMySQLDB(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory Factory) (*sql.DB, error) { + dsnStr, err := GenerateDSN(ctx, sinkURI, cfg, CreateMySQLDBConn) + if err != nil { + return nil, err + } + + db, err := CreateMySQLDBConn(ctx, dsnStr) + if err != nil { + return nil, err + } + + return db, nil +} + +func ParseURLAndCreateMySQLDBConn(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory Factory) (*sql.DB, error) { + +} + // CreateMySQLDBConn creates a mysql database connection with the given dsn. func CreateMySQLDBConn(ctx context.Context, dsnStr string) (*sql.DB, error) { + // TODO(wlwilliamx): 把这里改成生成多个 db 返回? db, err := sql.Open("mysql", dsnStr) if err != nil { return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") @@ -52,49 +70,46 @@ func CreateMySQLDBConn(ctx context.Context, dsnStr string) (*sql.DB, error) { return db, nil } +// TODO: 似乎关键是把这里给改了,改成返回多个 dsnStr。很多地方都用到了这里 +// TODO: comment // GenerateDSN generates the dsn with the given config. -func GenerateDSN(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory Factory) (dsnStr string, err error) { +func GenerateDSN(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory Factory) ([]string, error) { // dsn format of the driver: // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] - dsn, err := GenBasicDSN(sinkURI, cfg) + dsnCfg, err := GetDesinationCfg(sinkURI, cfg) if err != nil { - return "", err - } - - var testDB *sql.DB - testDB, err = GetTestDB(ctx, dsn, dbConnFactory) - if err != nil { - return + return nil, err } - defer testDB.Close() - // we use default sql mode for downstream because all dmls generated and ddls in ticdc - // are based on default sql mode. - dsn.Params["sql_mode"], err = dmutils.AdjustSQLModeCompatible(mysql.DefaultSQLMode) - if err != nil { - return - } - // NOTE: quote the string is necessary to avoid ambiguities. - dsn.Params["sql_mode"] = strconv.Quote(dsn.Params["sql_mode"]) + dsnStr := make([]string, len(dsnCfg)) + for i := 0; i < len(dsnCfg); i++ { + var testDB *sql.DB + // TODO: TestDB 这里冲突了,如果出错就直接返回了 + testDB, err = GetTestDB(ctx, dsnCfg[i], dbConnFactory) + if err != nil { + return nil, err + } + defer testDB.Close() - dsnStr, err = generateDSNByConfig(ctx, dsn, cfg, testDB) - if err != nil { - return - } + dsnStr[i], err = generateDSNByConfig(ctx, dsnCfg[i], cfg, testDB) + if err != nil { + return nil, err + } - // check if GBK charset is supported by downstream - var gbkSupported bool - gbkSupported, err = checkCharsetSupport(ctx, testDB, charset.CharsetGBK) - if err != nil { - return - } - if !gbkSupported { - log.Warn("GBK charset is not supported by the downstream. "+ - "Some types of DDLs may fail to execute", - zap.String("host", dsn.Addr)) + // check if GBK charset is supported by downstream + var gbkSupported bool + gbkSupported, err = checkCharsetSupport(ctx, testDB, charset.CharsetGBK) + if err != nil { + return nil, err + } + if !gbkSupported { + log.Warn("GBK charset is not supported by the downstream. "+ + "Some types of DDLs may fail to execute", + zap.String("host", dsnCfg[i].Addr)) + } } - return + return dsnStr, nil } func generateDSNByConfig( @@ -119,6 +134,16 @@ func generateDSNByConfig( // auto fetch max_allowed_packet on every new connection dsnCfg.Params["maxAllowedPacket"] = "0" + // we use default sql mode for downstream because all dmls generated and ddls in ticdc + // are based on default sql mode. + var err error + dsnCfg.Params["sql_mode"], err = dmutils.AdjustSQLModeCompatible(tmysql.DefaultSQLMode) + if err != nil { + return "", err + } + // NOTE: quote the string is necessary to avoid ambiguities. + dsnCfg.Params["sql_mode"] = strconv.Quote(dsnCfg.Params["sql_mode"]) + autoRandom, err := checkTiDBVariable(ctx, testDB, "allow_auto_random_explicit_insert", "1") if err != nil { return "", err @@ -234,8 +259,8 @@ func GetTestDB(ctx context.Context, dbConfig *dmysql.Config, dbConnFactory Facto return testDB, err } -// GenBasicDSN generates a basic DSN from the given config. -func GenBasicDSN(sinkURI *url.URL, cfg *Config) (*dmysql.Config, error) { +// GetDesinationCfg generates a basic DSN from the given config. +func GetDesinationCfg(sinkURI *url.URL, cfg *Config) ([]*dmysql.Config, error) { // dsn format of the driver: // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] username := sinkURI.User.Username() @@ -244,33 +269,46 @@ func GenBasicDSN(sinkURI *url.URL, cfg *Config) (*dmysql.Config, error) { } password, _ := sinkURI.User.Password() - hostName := sinkURI.Hostname() - port := sinkURI.Port() - if port == "" { - port = "4000" - } + hosts := getMultiAddressesInURL(sinkURI.Host) // This will handle the IPv6 address format. - var dsn *dmysql.Config - var err error - host := net.JoinHostPort(hostName, port) - dsnStr := fmt.Sprintf("%s:%s@tcp(%s)/%s", username, password, host, cfg.TLS) - if dsn, err = dmysql.ParseDSN(dsnStr); err != nil { - return nil, errors.Trace(err) - } + dsnCfg := make([]*dmysql.Config, len(hosts)) + for i := 0; i < len(dsnCfg); i++ { + var err error + dsnStr := fmt.Sprintf("%s:%s@tcp(%s)/%s", username, password, hosts[i], cfg.TLS) + if dsnCfg[i], err = dmysql.ParseDSN(dsnStr); err != nil { + return nil, errors.Trace(err) + } - // create test db used for parameter detection - // Refer https://github.com/go-sql-driver/mysql#parameters - if dsn.Params == nil { - dsn.Params = make(map[string]string, 1) + // create test db used for parameter detection + // Refer https://github.com/go-sql-driver/mysql#parameters + if dsnCfg[i].Params == nil { + dsnCfg[i].Params = make(map[string]string, 1) + } + if cfg.Timezone != "" { + dsnCfg[i].Params["time_zone"] = cfg.Timezone + } + dsnCfg[i].Params["readTimeout"] = cfg.ReadTimeout + dsnCfg[i].Params["writeTimeout"] = cfg.WriteTimeout + dsnCfg[i].Params["timeout"] = cfg.DialTimeout } - if cfg.Timezone != "" { - dsn.Params["time_zone"] = cfg.Timezone + return dsnCfg, nil +} + +func getMultiAddressesInURL(hostStr string) []string { + hosts := strings.Split(hostStr, ",") + for i := 0; i < len(hosts); i++ { + hostnameAndPort := strings.Split(hosts[i], ":") + if len(hostnameAndPort) == 1 { + hostnameAndPort = append(hostnameAndPort, "4000") + } else if len(hostnameAndPort) == 2 && hostnameAndPort[1] == "" { + hostnameAndPort[1] = "4000" + } else { + // TODO: invalid host, report error? + } + hosts[i] = hostnameAndPort[0] + ":" + hostnameAndPort[1] } - dsn.Params["readTimeout"] = cfg.ReadTimeout - dsn.Params["writeTimeout"] = cfg.WriteTimeout - dsn.Params["timeout"] = cfg.DialTimeout - return dsn, nil + return hosts } // CheckIfBDRModeIsSupported checks if the downstream supports BDR mode. @@ -342,7 +380,7 @@ func SetWriteSource(ctx context.Context, cfg *Config, txn *sql.Tx) error { _, err := txn.ExecContext(ctx, query) if err != nil { if mysqlErr, ok := errors.Cause(err).(*dmysql.MySQLError); ok && - mysqlErr.Number == mysql.ErrUnknownSystemVariable { + mysqlErr.Number == tmysql.ErrUnknownSystemVariable { return nil } return err diff --git a/tests/integration_tests/run_group.sh b/tests/integration_tests/run_group.sh index 991ba3d188a..a1ce80ead0a 100755 --- a/tests/integration_tests/run_group.sh +++ b/tests/integration_tests/run_group.sh @@ -42,7 +42,7 @@ groups=( # G04 'foreign_key ddl_puller_lag ddl_only_block_related_table changefeed_auto_stop' # G05 - 'charset_gbk ddl_manager multi_source' + 'charset_gbk ddl_manager multi_source' # # G06 'sink_retry changefeed_error ddl_sequence resourcecontrol' # G07 pulsar oauth2 authentication enabled From 7c8890fa7f16c5d5813257cdc8e3f054be883ad6 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Mon, 12 Aug 2024 16:14:44 +0800 Subject: [PATCH 02/50] feat(retry): add `WithPreExecutionWhenRetry` option to execute actions before each retry attempt Introduced the `WithPreExecutionWhenRetry` feature to allow a PreExecution action to be specified, which will be executed before each retry attempt, but only if the initial execution fails. This ensures that the PreExecution function is triggered before every retry following the first failure. If the initial execution is successful, no retry occurs, and the PreExecution function will not be executed. --- pkg/retry/options.go | 20 ++++++++++++++++++++ pkg/retry/retry_with_opt.go | 12 +++++++++++- 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/pkg/retry/options.go b/pkg/retry/options.go index 942f040253f..fc85881fb24 100644 --- a/pkg/retry/options.go +++ b/pkg/retry/options.go @@ -33,12 +33,19 @@ type Option func(*retryOptions) // IsRetryable checks the error is safe or worth to retry, eg. "context.Canceled" better not retry type IsRetryable func(error) bool +type PreExecution func() error + type retryOptions struct { totalRetryDuration time.Duration maxTries uint64 backoffBaseInMs float64 backoffCapInMs float64 isRetryable IsRetryable + + // preExecutionWhenRetry is executed before every retry attempt, except for the initial run. + // If the first execution fails and a retry is triggered, preExecutionWhenRetry() will be called + // before the main process is retried. This method ensures that it runs prior to each retry attempt. + preExecutionWhenRetry PreExecution } func newRetryOptions() *retryOptions { @@ -94,3 +101,16 @@ func WithIsRetryableErr(f IsRetryable) Option { } } } + +// WithPreExecutionWhenRetry allows you to specify a PreExecution action that will be executed +// before each retry attempt, but only if the initial execution fails. The PreExecution function +// will run before every retry following the first failure, since retries only occur after the +// initial attempt has failed. If the initial execution is successful, the PreExecution function +// will not be triggered, as no retry is necessary. +func WithPreExecutionWhenRetry(f PreExecution) Option { + return func(o *retryOptions) { + if f != nil { + o.preExecutionWhenRetry = f + } + } +} diff --git a/pkg/retry/retry_with_opt.go b/pkg/retry/retry_with_opt.go index 8f6bc12507c..50d20a83fca 100644 --- a/pkg/retry/retry_with_opt.go +++ b/pkg/retry/retry_with_opt.go @@ -53,13 +53,23 @@ func run(ctx context.Context, op Operation, retryOption *retryOptions) error { var start time.Time try := uint64(0) backOff := time.Duration(0) + isFirstRun := true for { + if isFirstRun { + isFirstRun = false + } else if retryOption.preExecutionWhenRetry != nil { + err := retryOption.preExecutionWhenRetry() + if err != nil { + return err + } + } + err := op() if err == nil { return nil } - if !retryOption.isRetryable(err) { + if retryOption.isRetryable != nil && !retryOption.isRetryable(err) { return err } From d856ea82b0935bd1f861dbe0c237412fb8c91a7e Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Mon, 12 Aug 2024 19:04:35 +0800 Subject: [PATCH 03/50] sync --- cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 28 +++--- cdc/sink/dmlsink/txn/mysql/mysql.go | 41 ++++----- cdc/sink/validator/validator.go | 2 +- cdc/syncpointstore/mysql_syncpoint_store.go | 28 ++---- pkg/sink/mysql/db_helper.go | 81 +++++++---------- pkg/sink/mysql/mysql_connector.go | 98 +++++++++++++++++++++ 6 files changed, 168 insertions(+), 110 deletions(-) create mode 100644 pkg/sink/mysql/mysql_connector.go diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index 074e3ef73f5..9685ac5a708 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -20,7 +20,7 @@ import ( "net/url" "time" - cerrors "github.com/pingcap/errors" + perrors "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" timodel "github.com/pingcap/tidb/pkg/parser/model" @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tiflow/cdc/sink/ddlsink" "github.com/pingcap/tiflow/cdc/sink/metrics" "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/errors" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/errorutil" "github.com/pingcap/tiflow/pkg/quotes" @@ -55,7 +54,8 @@ var _ ddlsink.Sink = (*DDLSink)(nil) // DDLSink is a sink that writes DDL events to MySQL. type DDLSink struct { // id indicates which processor (changefeed) this sink belongs to. - id model.ChangeFeedID + id model.ChangeFeedID + connector *pmysql.MySQLDBConnector // db is the database connection. db *sql.DB cfg *pmysql.Config @@ -77,29 +77,25 @@ func NewDDLSink( return nil, err } - dsnStr, err := pmysql.GenerateDSN(ctx, sinkURI, cfg, GetDBConnImpl) + // TODO: reserve the connector + connector, err := pmysql.NewMySQLDBConnector(ctx, cfg, sinkURI) if err != nil { return nil, err } - db, err := GetDBConnImpl(ctx, dsnStr) + cfg.IsTiDB, err = pmysql.CheckIsTiDB(ctx, connector.CurrentDB) if err != nil { return nil, err } - cfg.IsTiDB, err = pmysql.CheckIsTiDB(ctx, db) - if err != nil { - return nil, err - } - - cfg.IsWriteSourceExisted, err = pmysql.CheckIfBDRModeIsSupported(ctx, db) + cfg.IsWriteSourceExisted, err = pmysql.CheckIfBDRModeIsSupported(ctx, connector.CurrentDB) if err != nil { return nil, err } m := &DDLSink{ id: changefeedID, - db: db, + connector: connector, cfg: cfg, statistics: metrics.NewStatistics(ctx, changefeedID, sink.TxnSink), } @@ -140,7 +136,9 @@ func (m *DDLSink) execDDLWithMaxRetries(ctx context.Context, ddl *model.DDLEvent return err } return nil - }, retry.WithBackoffBaseDelay(pmysql.BackoffBaseDelay.Milliseconds()), + }, retry.WithPreExecutionWhenRetry(func() error { + return m.connector.SwitchToAvailableMySQLDB(ctx) + }), retry.WithBackoffBaseDelay(pmysql.BackoffBaseDelay.Milliseconds()), retry.WithBackoffMaxDelay(pmysql.BackoffMaxDelay.Milliseconds()), retry.WithMaxTries(defaultDDLMaxRetry), retry.WithIsRetryableErr(errorutil.IsRetryableDDLError)) @@ -212,7 +210,7 @@ func (m *DDLSink) execDDL(pctx context.Context, ddl *model.DDLEvent) error { // we try to set cdc write source for the ddl if err = pmysql.SetWriteSource(pctx, m.cfg, tx); err != nil { if rbErr := tx.Rollback(); rbErr != nil { - if errors.Cause(rbErr) != context.Canceled { + if perrors.Cause(rbErr) != context.Canceled { log.Error("Failed to rollback", zap.String("namespace", m.id.Namespace), zap.String("changefeed", m.id.ID), zap.Error(err)) @@ -235,7 +233,7 @@ func (m *DDLSink) execDDL(pctx context.Context, ddl *model.DDLEvent) error { zap.Duration("duration", time.Since(start)), zap.String("namespace", m.id.Namespace), zap.String("changefeed", m.id.ID), zap.Error(err)) - return cerror.WrapError(cerror.ErrMySQLTxnError, cerrors.WithMessage(err, fmt.Sprintf("Query info: %s; ", ddl.Query))) + return cerror.WrapError(cerror.ErrMySQLTxnError, perrors.WithMessage(err, fmt.Sprintf("Query info: %s; ", ddl.Query))) } log.Info("Exec DDL succeeded", zap.String("sql", ddl.Query), diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index 39ee3578e7d..f386b2aea91 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -59,9 +59,10 @@ const ( ) type mysqlBackend struct { - workerID int - changefeed string - db *sql.DB + workerID int + changefeed string + connector *pmysql.MySQLDBConnector + // db *sql.DB cfg *pmysql.Config dmlMaxRetry uint64 @@ -97,23 +98,17 @@ func NewMySQLBackends( return nil, err } - // TODO: 这里生成多个 dsnStr 返回 - dsnStr, err := pmysql.GenerateDSN(ctx, sinkURI, cfg, dbConnFactory) + connector, err := pmysql.NewMySQLDBConnector(ctx, cfg, sinkURI) if err != nil { return nil, err } - db, err := dbConnFactory(ctx, dsnStr) + cfg.IsTiDB, err = pmysql.CheckIsTiDB(ctx, connector.CurrentDB) if err != nil { return nil, err } - cfg.IsTiDB, err = pmysql.CheckIsTiDB(ctx, db) - if err != nil { - return nil, err - } - - cfg.IsWriteSourceExisted, err = pmysql.CheckIfBDRModeIsSupported(ctx, db) + cfg.IsWriteSourceExisted, err = pmysql.CheckIfBDRModeIsSupported(ctx, connector.CurrentDB) if err != nil { return nil, err } @@ -138,7 +133,7 @@ func NewMySQLBackends( cachePrepStmts := cfg.CachePrepStmts if cachePrepStmts { // query the size of the prepared statement cache on serverside - maxPreparedStmtCount, err := pmysql.QueryMaxPreparedStmtCount(ctx, db) + maxPreparedStmtCount, err := pmysql.QueryMaxPreparedStmtCount(ctx, connector.CurrentDB) if err != nil { return nil, err } @@ -168,7 +163,7 @@ func NewMySQLBackends( } var maxAllowedPacket int64 - maxAllowedPacket, err = pmysql.QueryMaxAllowedPacket(ctx, db) + maxAllowedPacket, err = pmysql.QueryMaxAllowedPacket(ctx, connector.CurrentDB) if err != nil { log.Warn("failed to query max_allowed_packet, use default value", zap.String("changefeed", changefeed), @@ -181,7 +176,7 @@ func NewMySQLBackends( backends = append(backends, &mysqlBackend{ workerID: i, changefeed: changefeed, - db: db, + connector: connector, cfg: cfg, dmlMaxRetry: defaultDMLMaxRetry, statistics: statistics, @@ -262,9 +257,9 @@ func (s *mysqlBackend) Close() (err error) { if s.stmtCache != nil { s.stmtCache.Purge() } - if s.db != nil { - err = s.db.Close() - s.db = nil + if s.connector.CurrentDB != nil { + err = s.connector.CurrentDB.Close() + s.connector = nil } return } @@ -674,7 +669,7 @@ func (s *mysqlBackend) sequenceExecute( if s.cachePrepStmts { if stmt, ok := s.stmtCache.Get(query); ok { prepStmt = stmt.(*sql.Stmt) - } else if stmt, err := s.db.Prepare(query); err == nil { + } else if stmt, err := s.connector.CurrentDB.Prepare(query); err == nil { prepStmt = stmt s.stmtCache.Add(query, stmt) } else { @@ -742,7 +737,7 @@ func (s *mysqlBackend) execDMLWithMaxRetries(pctx context.Context, dmls *prepare }) err := s.statistics.RecordBatchExecution(func() (int, int64, error) { - tx, err := s.db.BeginTx(pctx, nil) + tx, err := s.connector.CurrentDB.BeginTx(pctx, nil) if err != nil { return 0, 0, logDMLTxnErr( wrapMysqlTxnError(err), @@ -767,8 +762,6 @@ func (s *mysqlBackend) execDMLWithMaxRetries(pctx context.Context, dmls *prepare return 0, 0, err } - // mysql 8.0 5.7 - // If interplated SQL size exceeds maxAllowedPacket, mysql driver will // fall back to the sequantial way. // error can be ErrPrepareMulti, ErrBadConn etc. @@ -802,7 +795,9 @@ func (s *mysqlBackend) execDMLWithMaxRetries(pctx context.Context, dmls *prepare zap.Int("workerID", s.workerID), zap.Int("numOfRows", dmls.rowCount)) return nil - }, retry.WithBackoffBaseDelay(pmysql.BackoffBaseDelay.Milliseconds()), + }, retry.WithPreExecutionWhenRetry(func() error { + return s.connector.SwitchToAvailableMySQLDB(pctx) + }), retry.WithBackoffBaseDelay(pmysql.BackoffBaseDelay.Milliseconds()), retry.WithBackoffMaxDelay(pmysql.BackoffMaxDelay.Milliseconds()), retry.WithMaxTries(s.dmlMaxRetry), retry.WithIsRetryableErr(isRetryableDMLError)) diff --git a/cdc/sink/validator/validator.go b/cdc/sink/validator/validator.go index ef3b0842464..7c24e5c802a 100644 --- a/cdc/sink/validator/validator.go +++ b/cdc/sink/validator/validator.go @@ -122,7 +122,7 @@ func checkBDRMode(ctx context.Context, sinkURI *url.URL, replicaConfig *config.R if err != nil { return err } - dsn, err := pmysql.GetDesinationCfg(sinkURI, cfg) + dsn, err := pmysql.GetDSNCfg(sinkURI, cfg) if err != nil { return err } diff --git a/cdc/syncpointstore/mysql_syncpoint_store.go b/cdc/syncpointstore/mysql_syncpoint_store.go index 998fa89b252..4b8baf765ca 100644 --- a/cdc/syncpointstore/mysql_syncpoint_store.go +++ b/cdc/syncpointstore/mysql_syncpoint_store.go @@ -15,7 +15,6 @@ package syncpointstore import ( "context" - "database/sql" "fmt" "net/url" "time" @@ -32,7 +31,8 @@ import ( ) type mysqlSyncPointStore struct { - db *sql.DB + connector *mysql.MySQLDBConnector + // db *sql.DB clusterID string syncPointRetention time.Duration lastCleanSyncPointTime time.Time @@ -50,31 +50,13 @@ func newMySQLSyncPointStore( if err != nil { return nil, err } - getTestDb := func(ctx context.Context, dsnStr string) (*sql.DB, error) { - testDB, err := sql.Open("mysql", dsnStr) - if err != nil { - return nil, err - } - return testDB, nil - } - dsnStr, err := mysql.GenerateDSN(ctx, sinkURI, cfg, getTestDb) - if err != nil { - return nil, errors.Trace(err) - } - syncDB, err := sql.Open("mysql", dsnStr) - if err != nil { - return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") - } - err = syncDB.PingContext(ctx) - if err != nil { - return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") - } + connector, err := mysql.NewMySQLDBConnector(ctx, cfg, sinkURI) log.Info("Start mysql syncpoint sink", zap.String("changefeed", id.String())) return &mysqlSyncPointStore{ - db: syncDB, + connector: connector, clusterID: config.GetGlobalServerConfig().ClusterID, syncPointRetention: syncPointRetention, lastCleanSyncPointTime: time.Now(), @@ -201,6 +183,6 @@ func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context, } func (s *mysqlSyncPointStore) Close() error { - err := s.db.Close() + err := s.connector.CurrentDB.Close() return cerror.WrapError(cerror.ErrMySQLConnectionError, err) } diff --git a/pkg/sink/mysql/db_helper.go b/pkg/sink/mysql/db_helper.go index b351d6952ff..72ab62a1529 100644 --- a/pkg/sink/mysql/db_helper.go +++ b/pkg/sink/mysql/db_helper.go @@ -32,27 +32,8 @@ import ( "go.uber.org/zap" ) -func ConnectOneAvailableMySQLDB(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory Factory) (*sql.DB, error) { - dsnStr, err := GenerateDSN(ctx, sinkURI, cfg, CreateMySQLDBConn) - if err != nil { - return nil, err - } - - db, err := CreateMySQLDBConn(ctx, dsnStr) - if err != nil { - return nil, err - } - - return db, nil -} - -func ParseURLAndCreateMySQLDBConn(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory Factory) (*sql.DB, error) { - -} - // CreateMySQLDBConn creates a mysql database connection with the given dsn. func CreateMySQLDBConn(ctx context.Context, dsnStr string) (*sql.DB, error) { - // TODO(wlwilliamx): 把这里改成生成多个 db 返回? db, err := sql.Open("mysql", dsnStr) if err != nil { return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") @@ -70,43 +51,47 @@ func CreateMySQLDBConn(ctx context.Context, dsnStr string) (*sql.DB, error) { return db, nil } -// TODO: 似乎关键是把这里给改了,改成返回多个 dsnStr。很多地方都用到了这里 -// TODO: comment // GenerateDSN generates the dsn with the given config. -func GenerateDSN(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory Factory) ([]string, error) { - // dsn format of the driver: - // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] - dsnCfg, err := GetDesinationCfg(sinkURI, cfg) +func GenerateDSN(ctx context.Context, sinkURI *url.URL, cfg *Config) ([]string, error) { + dsnCfg, err := GetDSNCfg(sinkURI, cfg) if err != nil { return nil, err } - dsnStr := make([]string, len(dsnCfg)) + var oneOfErrs error + dsn := make([]string, len(dsnCfg)) for i := 0; i < len(dsnCfg); i++ { - var testDB *sql.DB - // TODO: TestDB 这里冲突了,如果出错就直接返回了 - testDB, err = GetTestDB(ctx, dsnCfg[i], dbConnFactory) + dsn[i], err = checkAndGenerateDSNByConfig(ctx, dsnCfg[i], cfg) if err != nil { - return nil, err + oneOfErrs = err } - defer testDB.Close() + } - dsnStr[i], err = generateDSNByConfig(ctx, dsnCfg[i], cfg, testDB) - if err != nil { - return nil, err - } + return dsn, oneOfErrs +} - // check if GBK charset is supported by downstream - var gbkSupported bool - gbkSupported, err = checkCharsetSupport(ctx, testDB, charset.CharsetGBK) - if err != nil { - return nil, err - } - if !gbkSupported { - log.Warn("GBK charset is not supported by the downstream. "+ - "Some types of DDLs may fail to execute", - zap.String("host", dsnCfg[i].Addr)) - } +func checkAndGenerateDSNByConfig(ctx context.Context, dsnCfg *dmysql.Config, cfg *Config) (string, error) { + testDB, err := GetTestDB(ctx, dsnCfg, CreateMySQLDBConn) + if err != nil { + return "", err + } + defer testDB.Close() + + // check if GBK charset is supported by downstream + var gbkSupported bool + gbkSupported, err = checkCharsetSupport(ctx, testDB, charset.CharsetGBK) + if err != nil { + return "", err + } + if !gbkSupported { + log.Warn("GBK charset is not supported by the downstream. "+ + "Some types of DDLs may fail to execute", + zap.String("host", dsnCfg.Addr)) + } + + dsnStr, err := generateDSNByConfig(ctx, dsnCfg, cfg, testDB) + if err != nil { + return "", err } return dsnStr, nil @@ -259,8 +244,8 @@ func GetTestDB(ctx context.Context, dbConfig *dmysql.Config, dbConnFactory Facto return testDB, err } -// GetDesinationCfg generates a basic DSN from the given config. -func GetDesinationCfg(sinkURI *url.URL, cfg *Config) ([]*dmysql.Config, error) { +// GetDSNCfg generates a basic DSN from the given config. +func GetDSNCfg(sinkURI *url.URL, cfg *Config) ([]*dmysql.Config, error) { // dsn format of the driver: // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] username := sinkURI.User.Username() diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go new file mode 100644 index 00000000000..7203ca363bf --- /dev/null +++ b/pkg/sink/mysql/mysql_connector.go @@ -0,0 +1,98 @@ +package mysql + +import ( + "context" + "database/sql" + "net/url" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "go.uber.org/zap" +) + +type MySQLDBConnector struct { + // The current DB connnection. + CurrentDB *sql.DB + + // The list of Data Source Names (DSNs), which are used as inputs for sql.Open(). + // The DSN format for passing to the database driver: + // [username[:password]@][net[(address)]]/dbname[?param1=value1&...¶mN=valueN] + dsnList []string + + // The index for the next connection attempt, + // used to try to find an available DB server in Round-Robin mode. + nextTry int +} + +// The sinkURI format: +// [scheme]://[user[:password]@][host[:port]][,host[:port]][,host[:port]][/path][?param1=value1¶mN=valueN] +// User must ensure that each address ([host[:port]]) in the sinkURI (if there are multiple addresses) +// is valid. +func NewMySQLDBConnector(ctx context.Context, cfg *Config, sinkURI *url.URL) (*MySQLDBConnector, error) { + if sinkURI.Scheme != "mysql" && sinkURI.Scheme != "tidb" { + return nil, errors.New("") + } + + // GenerateDSN function parses multiple addresses from the URL (if any) + // and generates a DSN (Data Source Name) for each one. + // For each DSN, the function attempts to create a connection and perform a Ping + // to verify its availability. + // If any of the DSNs are unavailable, the function immediately returns an error. + dsn, err := GenerateDSN(ctx, sinkURI, cfg) + if err != nil { + return nil, err + } + + connector := &MySQLDBConnector{dsnList: dsn, nextTry: 0} + + err = connector.SwitchToAvailableMySQLDB(ctx) + if err != nil { + return nil, err + } + + return connector, nil +} + +// SwitchToAvailableMySQLDB attempts to switch to an available MySQL-compatible database connection +// if the current connection is invalid, and it updates MySQLDBConnector.CurrentDB for user to use it. +// If there is only one DSN in MySQLDBConnector, switching is not possible. +func (c *MySQLDBConnector) SwitchToAvailableMySQLDB(ctx context.Context) error { + // If a connection has already been established and there is only one DSN (Data Source Name) available, + // it is not possible to connect a different DSN. Therefore, simply return from the function. + if len(c.dsnList) == 1 && c.CurrentDB != nil { + return nil + } + + if c.CurrentDB != nil { + // Check if the current connection is available; return immediately if it is. + err := c.CurrentDB.PingContext(ctx) + if err == nil { + return nil + } + + // The current connection has become invalid. Close this connection to free up resources, + // then attempt to establish a new connection using a different DSN. + closeErr := c.CurrentDB.Close() + if closeErr != nil { + log.Warn("close db failed", zap.Error(err)) + } + } + + var err error + for i := 0; i < len(c.dsnList); i++ { + if c.dsnList[c.nextTry] == "" { + continue + } + + db, err := CreateMySQLDBConn(ctx, c.dsnList[c.nextTry]) + c.nextTry = (c.nextTry + 1) % len(c.dsnList) + if err == nil { + c.CurrentDB = db + return nil + } + } + + return err +} + +// TODO: 设置每次切换的配置 From 120561fe9af2b1fb570a43f456e2dce9bf60d6a1 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 13 Aug 2024 16:16:35 +0800 Subject: [PATCH 04/50] feat(DDLSink): support multi mysql downstream addresses for DDL Sink --- cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index 9685ac5a708..65c7210c2aa 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -15,7 +15,6 @@ package mysql import ( "context" - "database/sql" "fmt" "net/url" "time" @@ -56,9 +55,7 @@ type DDLSink struct { // id indicates which processor (changefeed) this sink belongs to. id model.ChangeFeedID connector *pmysql.MySQLDBConnector - // db is the database connection. - db *sql.DB - cfg *pmysql.Config + cfg *pmysql.Config // statistics is the statistics of this sink. // We use it to record the DDL count. statistics *metrics.Statistics @@ -77,8 +74,7 @@ func NewDDLSink( return nil, err } - // TODO: reserve the connector - connector, err := pmysql.NewMySQLDBConnector(ctx, cfg, sinkURI) + connector, err := pmysql.NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, GetDBConnImpl) if err != nil { return nil, err } @@ -191,7 +187,7 @@ func (m *DDLSink) execDDL(pctx context.Context, ddl *model.DDLEvent) error { start := time.Now() log.Info("Start exec DDL", zap.String("DDL", ddl.Query), zap.Uint64("commitTs", ddl.CommitTs), zap.String("namespace", m.id.Namespace), zap.String("changefeed", m.id.ID)) - tx, err := m.db.BeginTx(ctx, nil) + tx, err := m.connector.CurrentDB.BeginTx(ctx, nil) if err != nil { return err } @@ -264,8 +260,8 @@ func (m *DDLSink) Close() { if m.statistics != nil { m.statistics.Close() } - if m.db != nil { - if err := m.db.Close(); err != nil { + if m.connector.CurrentDB != nil { + if err := m.connector.CurrentDB.Close(); err != nil { log.Warn("MySQL ddl sink close db wit error", zap.String("namespace", m.id.Namespace), zap.String("changefeed", m.id.ID), From 6b1ca91252f4f174306d92b6af803955d3374063 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 13 Aug 2024 18:34:09 +0800 Subject: [PATCH 05/50] feat(mysqlBackend): support multi mysql downstream addresses for DML Sink --- cdc/sink/dmlsink/txn/mysql/mysql.go | 41 +++++++++++++++-------------- 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index f386b2aea91..81abe2ed823 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -59,10 +59,9 @@ const ( ) type mysqlBackend struct { - workerID int - changefeed string - connector *pmysql.MySQLDBConnector - // db *sql.DB + workerID int + changefeed string + connector *pmysql.MySQLDBConnector cfg *pmysql.Config dmlMaxRetry uint64 @@ -98,7 +97,7 @@ func NewMySQLBackends( return nil, err } - connector, err := pmysql.NewMySQLDBConnector(ctx, cfg, sinkURI) + connector, err := pmysql.NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) if err != nil { return nil, err } @@ -113,21 +112,23 @@ func NewMySQLBackends( return nil, err } - // By default, cache-prep-stmts=true, an LRU cache is used for prepared statements, - // two connections are required to process a transaction. - // The first connection is held in the tx variable, which is used to manage the transaction. - // The second connection is requested through a call to s.db.Prepare - // in case of a cache miss for the statement query. - // The connection pool for CDC is configured with a static size, equal to the number of workers. - // CDC may hang at the "Get Connection" call is due to the limited size of the connection pool. - // When the connection pool is small, - // the chance of all connections being active at the same time increases, - // leading to exhaustion of available connections and a hang at the "Get Connection" call. - // This issue is less likely to occur when the connection pool is larger, - // as there are more connections available for use. - // Adding an extra connection to the connection pool solves the connection exhaustion issue. - db.SetMaxIdleConns(cfg.WorkerCount + 1) - db.SetMaxOpenConns(cfg.WorkerCount + 1) + connector.ConfigureDBWhenSwitch(func() { + // By default, cache-prep-stmts=true, an LRU cache is used for prepared statements, + // two connections are required to process a transaction. + // The first connection is held in the tx variable, which is used to manage the transaction. + // The second connection is requested through a call to s.db.Prepare + // in case of a cache miss for the statement query. + // The connection pool for CDC is configured with a static size, equal to the number of workers. + // CDC may hang at the "Get Connection" call is due to the limited size of the connection pool. + // When the connection pool is small, + // the chance of all connections being active at the same time increases, + // leading to exhaustion of available connections and a hang at the "Get Connection" call. + // This issue is less likely to occur when the connection pool is larger, + // as there are more connections available for use. + // Adding an extra connection to the connection pool solves the connection exhaustion issue. + connector.CurrentDB.SetMaxIdleConns(cfg.WorkerCount + 1) + connector.CurrentDB.SetMaxOpenConns(cfg.WorkerCount + 1) + }, true) // Inherit the default value of the prepared statement cache from the SinkURI Options cachePrepStmts := cfg.CachePrepStmts From ccb58ff891d53ba6d99e9cd899f2076b2bc68421 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 13 Aug 2024 19:11:53 +0800 Subject: [PATCH 06/50] feat(mysqlSyncPointStore): support multi mysql downstream addresses for MySQL Syncpoint Store --- cdc/syncpointstore/mysql_syncpoint_store.go | 201 +++++++++++--------- 1 file changed, 106 insertions(+), 95 deletions(-) diff --git a/cdc/syncpointstore/mysql_syncpoint_store.go b/cdc/syncpointstore/mysql_syncpoint_store.go index 4b8baf765ca..fbd2bab06e0 100644 --- a/cdc/syncpointstore/mysql_syncpoint_store.go +++ b/cdc/syncpointstore/mysql_syncpoint_store.go @@ -26,13 +26,13 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/errorutil" "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/retry" "github.com/pingcap/tiflow/pkg/sink/mysql" "go.uber.org/zap" ) type mysqlSyncPointStore struct { - connector *mysql.MySQLDBConnector - // db *sql.DB + connector *mysql.MySQLDBConnector clusterID string syncPointRetention time.Duration lastCleanSyncPointTime time.Time @@ -52,6 +52,9 @@ func newMySQLSyncPointStore( } connector, err := mysql.NewMySQLDBConnector(ctx, cfg, sinkURI) + if err != nil { + return nil, err + } log.Info("Start mysql syncpoint sink", zap.String("changefeed", id.String())) @@ -64,122 +67,130 @@ func newMySQLSyncPointStore( } func (s *mysqlSyncPointStore) CreateSyncTable(ctx context.Context) error { - database := filter.TiCDCSystemSchema - tx, err := s.db.BeginTx(ctx, nil) - if err != nil { - log.Error("create sync table: begin Tx fail", zap.Error(err)) - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "create sync table: begin Tx fail;")) - } - _, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database) - if err != nil { - err2 := tx.Rollback() - if err2 != nil { - log.Error("failed to create syncpoint table", zap.Error(err2)) + return retry.Do(ctx, func() error { + database := filter.TiCDCSystemSchema + tx, err := s.connector.CurrentDB.BeginTx(ctx, nil) + if err != nil { + log.Error("create sync table: begin Tx fail", zap.Error(err)) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "create sync table: begin Tx fail;")) } - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) - } - _, err = tx.Exec("USE " + database) - if err != nil { - err2 := tx.Rollback() - if err2 != nil { - log.Error("failed to create syncpoint table", zap.Error(err2)) + _, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database) + if err != nil { + err2 := tx.Rollback() + if err2 != nil { + log.Error("failed to create syncpoint table", zap.Error(err2)) + } + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) } - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) - } - query := `CREATE TABLE IF NOT EXISTS %s - ( - ticdc_cluster_id varchar (255), - changefeed varchar(255), - primary_ts varchar(18), - secondary_ts varchar(18), - created_at timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, - INDEX (created_at), - PRIMARY KEY (changefeed, primary_ts) - );` - query = fmt.Sprintf(query, filter.SyncPointTable) - _, err = tx.Exec(query) - if err != nil { - err2 := tx.Rollback() - if err2 != nil { - log.Error("failed to create syncpoint table", zap.Error(err2)) + _, err = tx.Exec("USE " + database) + if err != nil { + err2 := tx.Rollback() + if err2 != nil { + log.Error("failed to create syncpoint table", zap.Error(err2)) + } + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) + } + query := `CREATE TABLE IF NOT EXISTS %s + ( + ticdc_cluster_id varchar (255), + changefeed varchar(255), + primary_ts varchar(18), + secondary_ts varchar(18), + created_at timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP, + INDEX (created_at), + PRIMARY KEY (changefeed, primary_ts) + );` + query = fmt.Sprintf(query, filter.SyncPointTable) + _, err = tx.Exec(query) + if err != nil { + err2 := tx.Rollback() + if err2 != nil { + log.Error("failed to create syncpoint table", zap.Error(err2)) + } + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) } + err = tx.Commit() return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) - } - err = tx.Commit() - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) + }, retry.WithPreExecutionWhenRetry(func() error { + return s.connector.SwitchToAvailableMySQLDB(ctx) + }), retry.WithMaxTries(2)) } func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context, id model.ChangeFeedID, checkpointTs uint64, ) error { - tx, err := s.db.BeginTx(ctx, nil) - if err != nil { - log.Error("sync table: begin Tx fail", zap.Error(err)) - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "sync table: begin Tx fail;")) - } - 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", zap.String("changefeed", id.String())) - err2 := tx.Rollback() - if err2 != nil { - log.Error("failed to write syncpoint table", zap.Error(err)) + return retry.Do(ctx, func() error { + tx, err := s.connector.CurrentDB.BeginTx(ctx, nil) + if err != nil { + log.Error("sync table: begin Tx fail", zap.Error(err)) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "sync table: begin Tx fail;")) } - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) - } - // insert ts map - query := "insert ignore into " + filter.TiCDCSystemSchema + "." + filter.SyncPointTable + - "(ticdc_cluster_id, changefeed, primary_ts, secondary_ts) VALUES (?,?,?,?)" - _, err = tx.Exec(query, s.clusterID, id.ID, checkpointTs, secondaryTs) - if err != nil { - err2 := tx.Rollback() - if err2 != nil { - log.Error("failed to write syncpoint table", zap.Error(err2)) + 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", zap.String("changefeed", id.String())) + err2 := tx.Rollback() + if err2 != nil { + log.Error("failed to write syncpoint table", zap.Error(err)) + } + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) } - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) - } - - // set global tidb_external_ts to secondary ts - // TiDB supports tidb_external_ts system variable since v6.4.0. - query = fmt.Sprintf("set global tidb_external_ts = %s", secondaryTs) - _, err = tx.Exec(query) - if err != nil { - if errorutil.IsSyncPointIgnoreError(err) { - // TODO(dongmen): to confirm if we need to log this error. - log.Warn("set global external ts failed, ignore this error", zap.Error(err)) - } else { + // insert ts map + query := "insert ignore into " + filter.TiCDCSystemSchema + "." + filter.SyncPointTable + + "(ticdc_cluster_id, changefeed, primary_ts, secondary_ts) VALUES (?,?,?,?)" + _, err = tx.Exec(query, s.clusterID, id.ID, checkpointTs, secondaryTs) + if err != nil { err2 := tx.Rollback() if err2 != nil { log.Error("failed to write syncpoint table", zap.Error(err2)) } return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) } - } - // clean stale ts map in downstream - if time.Since(s.lastCleanSyncPointTime) >= s.syncPointRetention { - query = fmt.Sprintf( - "DELETE IGNORE FROM "+ - filter.TiCDCSystemSchema+"."+ - filter.SyncPointTable+ - " WHERE ticdc_cluster_id = '%s' and changefeed = '%s' and created_at < (NOW() - INTERVAL %.2f SECOND)", - s.clusterID, - id.ID, - s.syncPointRetention.Seconds()) + // set global tidb_external_ts to secondary ts + // TiDB supports tidb_external_ts system variable since v6.4.0. + query = fmt.Sprintf("set global tidb_external_ts = %s", secondaryTs) _, err = tx.Exec(query) if err != nil { - // It is ok to ignore the error, since it will not affect the correctness of the system, - // and no any business logic depends on this behavior, so we just log the error. - log.Error("failed to clean syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err))) - } else { - s.lastCleanSyncPointTime = time.Now() + if errorutil.IsSyncPointIgnoreError(err) { + // TODO(dongmen): to confirm if we need to log this error. + log.Warn("set global external ts failed, ignore this error", zap.Error(err)) + } else { + err2 := tx.Rollback() + if err2 != nil { + log.Error("failed to write syncpoint table", zap.Error(err2)) + } + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) + } + } + + // clean stale ts map in downstream + if time.Since(s.lastCleanSyncPointTime) >= s.syncPointRetention { + query = fmt.Sprintf( + "DELETE IGNORE FROM "+ + filter.TiCDCSystemSchema+"."+ + filter.SyncPointTable+ + " WHERE ticdc_cluster_id = '%s' and changefeed = '%s' and created_at < (NOW() - INTERVAL %.2f SECOND)", + s.clusterID, + id.ID, + s.syncPointRetention.Seconds()) + _, err = tx.Exec(query) + if err != nil { + // It is ok to ignore the error, since it will not affect the correctness of the system, + // and no any business logic depends on this behavior, so we just log the error. + log.Error("failed to clean syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err))) + } else { + s.lastCleanSyncPointTime = time.Now() + } } - } - err = tx.Commit() - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) + err = tx.Commit() + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) + }, retry.WithPreExecutionWhenRetry(func() error { + return s.connector.SwitchToAvailableMySQLDB(ctx) + }), retry.WithMaxTries(2)) } func (s *mysqlSyncPointStore) Close() error { From 735b08934d053dc9869f580cfbbcd1ac30f4676b Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 13 Aug 2024 19:31:40 +0800 Subject: [PATCH 07/50] feat(Observer): support multi mysql downstream addresses for Observer --- pkg/sink/observer/observer.go | 18 +++++------ pkg/sink/observer/tidb.go | 60 +++++++++++++++++++---------------- 2 files changed, 41 insertions(+), 37 deletions(-) diff --git a/pkg/sink/observer/observer.go b/pkg/sink/observer/observer.go index 64eb31a4862..5bad1530f67 100644 --- a/pkg/sink/observer/observer.go +++ b/pkg/sink/observer/observer.go @@ -81,25 +81,23 @@ func NewObserver( return nil, err } - dsnStr, err := pmysql.GenerateDSN(ctx, sinkURI, cfg, options.dbConnFactory) + connector, err := pmysql.NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, options.dbConnFactory) if err != nil { return nil, err } - db, err := options.dbConnFactory(ctx, dsnStr) - if err != nil { - return nil, err - } - db.SetMaxIdleConns(2) - db.SetMaxOpenConns(2) + connector.ConfigureDBWhenSwitch(func() { + connector.CurrentDB.SetMaxIdleConns(2) + connector.CurrentDB.SetMaxOpenConns(2) + }, true) - isTiDB, err := pmysql.CheckIsTiDB(ctx, db) + isTiDB, err := pmysql.CheckIsTiDB(ctx, connector.CurrentDB) if err != nil { return nil, err } if isTiDB { - return NewTiDBObserver(db), nil + return NewTiDBObserver(connector), nil } - _ = db.Close() + _ = connector.CurrentDB.Close() return NewDummyObserver(), nil } return &observerAgent{creator: creator}, nil diff --git a/pkg/sink/observer/tidb.go b/pkg/sink/observer/tidb.go index 8a39faca836..64200d46492 100644 --- a/pkg/sink/observer/tidb.go +++ b/pkg/sink/observer/tidb.go @@ -20,6 +20,8 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/sink/mysql" "go.uber.org/zap" ) @@ -106,14 +108,14 @@ var ( // TiDBObserver is a tidb performance observer. It's not thread-safe. type TiDBObserver struct { - db *sql.DB + connector *mysql.MySQLDBConnector } // Tick implements Observer func (o *TiDBObserver) Tick(ctx context.Context) error { m1 := make([]*tidbConnIdleDuration, 0) if err := queryMetrics[tidbConnIdleDuration]( - ctx, o.db, queryConnIdleDurationStmt, &m1); err != nil { + ctx, o.connector, queryConnIdleDurationStmt, &m1); err != nil { return err } for _, m := range m1 { @@ -129,7 +131,7 @@ func (o *TiDBObserver) Tick(ctx context.Context) error { m2 := make([]*tidbConnCount, 0) if err := queryMetrics[tidbConnCount]( - ctx, o.db, queryConnCountStmt, &m2); err != nil { + ctx, o.connector, queryConnCountStmt, &m2); err != nil { return err } for _, m := range m2 { @@ -141,7 +143,7 @@ func (o *TiDBObserver) Tick(ctx context.Context) error { m3 := make([]*tidbQueryDuration, 0) if err := queryMetrics[tidbQueryDuration]( - ctx, o.db, queryQueryDurationStmt, &m3); err != nil { + ctx, o.connector, queryQueryDurationStmt, &m3); err != nil { return err } for _, m := range m3 { @@ -153,7 +155,7 @@ func (o *TiDBObserver) Tick(ctx context.Context) error { m4 := make([]*tidbTxnDuration, 0) if err := queryMetrics[tidbTxnDuration]( - ctx, o.db, queryTxnDurationStmt, &m4); err != nil { + ctx, o.connector, queryTxnDurationStmt, &m4); err != nil { return err } for _, m := range m4 { @@ -168,13 +170,13 @@ func (o *TiDBObserver) Tick(ctx context.Context) error { // Close implements Observer func (o *TiDBObserver) Close() error { - return o.db.Close() + return o.connector.CurrentDB.Close() } // NewTiDBObserver creates a new TiDBObserver instance -func NewTiDBObserver(db *sql.DB) *TiDBObserver { +func NewTiDBObserver(connector *mysql.MySQLDBConnector) *TiDBObserver { return &TiDBObserver{ - db: db, + connector: connector, } } @@ -232,26 +234,30 @@ type metricColumnIface[T metricColumnImpl] interface { } func queryMetrics[T metricColumnImpl, F metricColumnIface[T]]( - ctx context.Context, db *sql.DB, stmt string, metrics *[]F, + ctx context.Context, connector *mysql.MySQLDBConnector, stmt string, metrics *[]F, ) error { - rows, err := db.QueryContext(ctx, stmt) - if err != nil { - return errors.WrapError(errors.ErrMySQLQueryError, err) - } - defer func() { - if err := rows.Close(); err != nil { - log.Warn("query metrics close rows failed", zap.Error(err)) - } - if rows.Err() != nil { - log.Warn("query metrics rows has error", zap.Error(rows.Err())) - } - }() - for rows.Next() { - var m F = new(T) - if err := rows.Scan(m.columns()...); err != nil { + return retry.Do(ctx, func() error { + rows, err := connector.CurrentDB.QueryContext(ctx, stmt) + if err != nil { return errors.WrapError(errors.ErrMySQLQueryError, err) } - *metrics = append(*metrics, m) - } - return nil + defer func() { + if err := rows.Close(); err != nil { + log.Warn("query metrics close rows failed", zap.Error(err)) + } + if rows.Err() != nil { + log.Warn("query metrics rows has error", zap.Error(rows.Err())) + } + }() + for rows.Next() { + var m F = new(T) + if err := rows.Scan(m.columns()...); err != nil { + return errors.WrapError(errors.ErrMySQLQueryError, err) + } + *metrics = append(*metrics, m) + } + return nil + }, retry.WithPreExecutionWhenRetry(func() error { + return connector.SwitchToAvailableMySQLDB(ctx) + }), retry.WithMaxTries(2)) } From 4e826ce72fce56a83d44cda9b66df8a916f548ac Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 13 Aug 2024 19:35:29 +0800 Subject: [PATCH 08/50] refactor(Observer): remove a duplicate import --- pkg/sink/observer/observer.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/sink/observer/observer.go b/pkg/sink/observer/observer.go index 5bad1530f67..549f05b7c7b 100644 --- a/pkg/sink/observer/observer.go +++ b/pkg/sink/observer/observer.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/errors" - cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink" pmysql "github.com/pingcap/tiflow/pkg/sink/mysql" ) @@ -67,7 +66,7 @@ func NewObserver( sinkURI, err := url.Parse(sinkURIStr) if err != nil { - return nil, cerror.WrapError(cerror.ErrSinkURIInvalid, err) + return nil, errors.WrapError(errors.ErrSinkURIInvalid, err) } scheme := strings.ToLower(sinkURI.Scheme) From b4d9191207334d2723c29eb9fd6fe611ad1a843f Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Thu, 15 Aug 2024 00:36:52 +0800 Subject: [PATCH 09/50] WIP --- cdc/sink/validator/validator.go | 4 +-- pkg/sink/mysql/db_helper.go | 45 ++++++++++++------------ pkg/sink/mysql/mock_db.go | 8 ++--- pkg/sink/mysql/mysql_connector.go | 58 +++++++++++++++++++++++++------ pkg/sink/observer/tidb_test.go | 49 +++++++++++++++++++++++++- 5 files changed, 125 insertions(+), 39 deletions(-) diff --git a/cdc/sink/validator/validator.go b/cdc/sink/validator/validator.go index 7c24e5c802a..3b6f88ebdd5 100644 --- a/cdc/sink/validator/validator.go +++ b/cdc/sink/validator/validator.go @@ -122,11 +122,11 @@ func checkBDRMode(ctx context.Context, sinkURI *url.URL, replicaConfig *config.R if err != nil { return err } - dsn, err := pmysql.GetDSNCfg(sinkURI, cfg) + dsnList, err := pmysql.GetDSNCfgs(sinkURI, cfg) if err != nil { return err } - testDB, err := pmysql.GetTestDB(ctx, dsn, pmysql.CreateMySQLDBConn) + testDB, err := pmysql.GetTestDB(ctx, dsnList[0], pmysql.CreateMySQLDBConn) if err != nil { return err } diff --git a/pkg/sink/mysql/db_helper.go b/pkg/sink/mysql/db_helper.go index 72ab62a1529..fef87ee9a44 100644 --- a/pkg/sink/mysql/db_helper.go +++ b/pkg/sink/mysql/db_helper.go @@ -51,17 +51,17 @@ func CreateMySQLDBConn(ctx context.Context, dsnStr string) (*sql.DB, error) { return db, nil } -// GenerateDSN generates the dsn with the given config. -func GenerateDSN(ctx context.Context, sinkURI *url.URL, cfg *Config) ([]string, error) { - dsnCfg, err := GetDSNCfg(sinkURI, cfg) +// GenerateDSNs generates the DSN with the given config. +func GenerateDSNs(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory Factory) ([]string, error) { + dsnCfgs, err := GetDSNCfgs(sinkURI, cfg) if err != nil { return nil, err } var oneOfErrs error - dsn := make([]string, len(dsnCfg)) - for i := 0; i < len(dsnCfg); i++ { - dsn[i], err = checkAndGenerateDSNByConfig(ctx, dsnCfg[i], cfg) + dsn := make([]string, len(dsnCfgs)) + for i := 0; i < len(dsnCfgs); i++ { + dsn[i], err = checkAndGenerateDSNByConfig(ctx, dsnCfgs[i], cfg, dbConnFactory) if err != nil { oneOfErrs = err } @@ -70,8 +70,8 @@ func GenerateDSN(ctx context.Context, sinkURI *url.URL, cfg *Config) ([]string, return dsn, oneOfErrs } -func checkAndGenerateDSNByConfig(ctx context.Context, dsnCfg *dmysql.Config, cfg *Config) (string, error) { - testDB, err := GetTestDB(ctx, dsnCfg, CreateMySQLDBConn) +func checkAndGenerateDSNByConfig(ctx context.Context, dsnCfg *dmysql.Config, cfg *Config, dbConnFactory Factory) (string, error) { + testDB, err := GetTestDB(ctx, dsnCfg, dbConnFactory) if err != nil { return "", err } @@ -244,8 +244,9 @@ func GetTestDB(ctx context.Context, dbConfig *dmysql.Config, dbConnFactory Facto return testDB, err } -// GetDSNCfg generates a basic DSN from the given config. -func GetDSNCfg(sinkURI *url.URL, cfg *Config) ([]*dmysql.Config, error) { +// TODO: update comments +// GetDSNCfgs generates basic DSNs from the given config. +func GetDSNCfgs(sinkURI *url.URL, cfg *Config) ([]*dmysql.Config, error) { // dsn format of the driver: // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] username := sinkURI.User.Username() @@ -257,27 +258,27 @@ func GetDSNCfg(sinkURI *url.URL, cfg *Config) ([]*dmysql.Config, error) { hosts := getMultiAddressesInURL(sinkURI.Host) // This will handle the IPv6 address format. - dsnCfg := make([]*dmysql.Config, len(hosts)) - for i := 0; i < len(dsnCfg); i++ { + dsnCfgs := make([]*dmysql.Config, len(hosts)) + for i := 0; i < len(dsnCfgs); i++ { var err error dsnStr := fmt.Sprintf("%s:%s@tcp(%s)/%s", username, password, hosts[i], cfg.TLS) - if dsnCfg[i], err = dmysql.ParseDSN(dsnStr); err != nil { + if dsnCfgs[i], err = dmysql.ParseDSN(dsnStr); err != nil { return nil, errors.Trace(err) } // create test db used for parameter detection // Refer https://github.com/go-sql-driver/mysql#parameters - if dsnCfg[i].Params == nil { - dsnCfg[i].Params = make(map[string]string, 1) + if dsnCfgs[i].Params == nil { + dsnCfgs[i].Params = make(map[string]string, 1) } if cfg.Timezone != "" { - dsnCfg[i].Params["time_zone"] = cfg.Timezone + dsnCfgs[i].Params["time_zone"] = cfg.Timezone } - dsnCfg[i].Params["readTimeout"] = cfg.ReadTimeout - dsnCfg[i].Params["writeTimeout"] = cfg.WriteTimeout - dsnCfg[i].Params["timeout"] = cfg.DialTimeout + dsnCfgs[i].Params["readTimeout"] = cfg.ReadTimeout + dsnCfgs[i].Params["writeTimeout"] = cfg.WriteTimeout + dsnCfgs[i].Params["timeout"] = cfg.DialTimeout } - return dsnCfg, nil + return dsnCfgs, nil } func getMultiAddressesInURL(hostStr string) []string { @@ -285,11 +286,11 @@ func getMultiAddressesInURL(hostStr string) []string { for i := 0; i < len(hosts); i++ { hostnameAndPort := strings.Split(hosts[i], ":") if len(hostnameAndPort) == 1 { + // If only the hostname is provided, add the default port 4000. hostnameAndPort = append(hostnameAndPort, "4000") } else if len(hostnameAndPort) == 2 && hostnameAndPort[1] == "" { + // If both the hostname and port are provided, but the port is empty, then set the port to the default port 4000. hostnameAndPort[1] = "4000" - } else { - // TODO: invalid host, report error? } hosts[i] = hostnameAndPort[0] + ":" + hostnameAndPort[1] } diff --git a/pkg/sink/mysql/mock_db.go b/pkg/sink/mysql/mock_db.go index 67ba74c1901..d54bf15848a 100644 --- a/pkg/sink/mysql/mock_db.go +++ b/pkg/sink/mysql/mock_db.go @@ -27,6 +27,10 @@ func MockTestDB() (*sql.DB, error) { return nil, err } + mock.ExpectQuery("select character_set_name from information_schema.character_sets " + + "where character_set_name = 'gbk';").WillReturnRows( + sqlmock.NewRows([]string{"character_set_name"}).AddRow("gbk"), + ) columns := []string{"Variable_name", "Value"} mock.ExpectQuery("show session variables like 'allow_auto_random_explicit_insert';").WillReturnRows( sqlmock.NewRows(columns).AddRow("allow_auto_random_explicit_insert", "0"), @@ -47,10 +51,6 @@ func MockTestDB() (*sql.DB, error) { sqlmock.NewRows(columns). AddRow("tidb_enable_external_ts_read", "OFF"), ) - mock.ExpectQuery("select character_set_name from information_schema.character_sets " + - "where character_set_name = 'gbk';").WillReturnRows( - sqlmock.NewRows([]string{"character_set_name"}).AddRow("gbk"), - ) mock.ExpectQuery("select tidb_version()"). WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) mock.ExpectQuery("select tidb_version()"). diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go index 7203ca363bf..162889c3185 100644 --- a/pkg/sink/mysql/mysql_connector.go +++ b/pkg/sink/mysql/mysql_connector.go @@ -5,13 +5,16 @@ import ( "database/sql" "net/url" - "github.com/pingcap/errors" "github.com/pingcap/log" "go.uber.org/zap" ) +// MySQLDBConnector manages the database connection, handling reconnections +// in case of connection failures when calling SwitchToAvailableMySQLDB method. +// To execute SQL queries or interact with the database, use the CurrentDB field of +// MySQLDBConnector to access the underlying *sql.DB instance, i.e., MySQLDBConnector.CurrentDB. type MySQLDBConnector struct { - // The current DB connnection. + // Current connnection to a MySQL-compatible DB. CurrentDB *sql.DB // The list of Data Source Names (DSNs), which are used as inputs for sql.Open(). @@ -22,15 +25,32 @@ type MySQLDBConnector struct { // The index for the next connection attempt, // used to try to find an available DB server in Round-Robin mode. nextTry int + + // dbConnFactory is a Factory function for MySQLDBConnector that creates database connections. + dbConnFactory Factory + + // configureDBWhenSwitch stores the function that will be automatically invoked + // to configure a new MySQL connection after switching to it using the + // SwitchToAvailableMySQLDB function. This function will be set by the + // ConfigureDBWhenSwitch method and can be applied to both newly established + // connections and the current active connection if required. + configureDBWhenSwitch ConfigureDBWhenSwitch +} + +type ConfigureDBWhenSwitch func() + +func NewMySQLDBConnector(ctx context.Context, cfg *Config, sinkURI *url.URL) (*MySQLDBConnector, error) { + return NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, CreateMySQLDBConn) } +// New a MySQLDBConnector by the given factory function for creating DB connection. // The sinkURI format: // [scheme]://[user[:password]@][host[:port]][,host[:port]][,host[:port]][/path][?param1=value1¶mN=valueN] // User must ensure that each address ([host[:port]]) in the sinkURI (if there are multiple addresses) -// is valid. -func NewMySQLDBConnector(ctx context.Context, cfg *Config, sinkURI *url.URL) (*MySQLDBConnector, error) { - if sinkURI.Scheme != "mysql" && sinkURI.Scheme != "tidb" { - return nil, errors.New("") +// is valid, otherwise returns an error. +func NewMySQLDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *url.URL, dbConnFactory Factory) (*MySQLDBConnector, error) { + if dbConnFactory == nil { + dbConnFactory = CreateMySQLDBConn } // GenerateDSN function parses multiple addresses from the URL (if any) @@ -38,12 +58,12 @@ func NewMySQLDBConnector(ctx context.Context, cfg *Config, sinkURI *url.URL) (*M // For each DSN, the function attempts to create a connection and perform a Ping // to verify its availability. // If any of the DSNs are unavailable, the function immediately returns an error. - dsn, err := GenerateDSN(ctx, sinkURI, cfg) + dsnList, err := GenerateDSNs(ctx, sinkURI, cfg, dbConnFactory) if err != nil { return nil, err } - connector := &MySQLDBConnector{dsnList: dsn, nextTry: 0} + connector := &MySQLDBConnector{dsnList: dsnList, nextTry: 0, dbConnFactory: dbConnFactory} err = connector.SwitchToAvailableMySQLDB(ctx) if err != nil { @@ -84,10 +104,13 @@ func (c *MySQLDBConnector) SwitchToAvailableMySQLDB(ctx context.Context) error { continue } - db, err := CreateMySQLDBConn(ctx, c.dsnList[c.nextTry]) + db, err := c.dbConnFactory(ctx, c.dsnList[c.nextTry]) c.nextTry = (c.nextTry + 1) % len(c.dsnList) if err == nil { c.CurrentDB = db + if c.configureDBWhenSwitch != nil { + c.configureDBWhenSwitch() + } return nil } } @@ -95,4 +118,19 @@ func (c *MySQLDBConnector) SwitchToAvailableMySQLDB(ctx context.Context) error { return err } -// TODO: 设置每次切换的配置 +// ConfigureDBWhenSwitch allows for automatic configuration of a new MySQL connection +// when switching to an available connection using the SwitchToAvailableMySQLDB function. +// By providing the function `f` as an argument, it ensures that any newly established +// connection is automatically configured after the switch. Additionally, if the existing +// MySQLDBConnector.CurrentDB also requires configuration, you can set +// `needConfigureCurrentDB` to true, and this function will automatically apply +// the configuration function `f` to it as well. +func (c *MySQLDBConnector) ConfigureDBWhenSwitch(f ConfigureDBWhenSwitch, needConfigureCurrentDB bool) { + if f == nil { + return + } + c.configureDBWhenSwitch = f + if needConfigureCurrentDB { + c.configureDBWhenSwitch() + } +} diff --git a/pkg/sink/observer/tidb_test.go b/pkg/sink/observer/tidb_test.go index 7303209e457..8a113fa4c95 100644 --- a/pkg/sink/observer/tidb_test.go +++ b/pkg/sink/observer/tidb_test.go @@ -15,9 +15,12 @@ package observer import ( "context" + "database/sql" + "net/url" "testing" "github.com/DATA-DOG/go-sqlmock" + "github.com/pingcap/tiflow/pkg/sink/mysql" "github.com/stretchr/testify/require" ) @@ -27,6 +30,7 @@ func TestTiDBObserver(t *testing.T) { db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) require.NoError(t, err) + mockSQLForPassCreateConnCheck(mock) mock.ExpectQuery(queryConnIdleDurationStmt). WillReturnRows( sqlmock.NewRows( @@ -68,10 +72,53 @@ func TestTiDBObserver(t *testing.T) { ) mock.ExpectClose() + mockGetDBConnImpl := func(ctx context.Context, dsnStr string) (*sql.DB, error) { + return db, nil + } + cfg := mysql.NewConfig() + sinkURIStr := "mysql://127.0.0.1:3306/" + sinkURI, err := url.Parse(sinkURIStr) + require.NoError(t, err) ctx := context.Background() - observer := NewTiDBObserver(db) + connector, err := mysql.NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, mockGetDBConnImpl) + require.NoError(t, err) + + observer := NewTiDBObserver(connector) err = observer.Tick(ctx) require.NoError(t, err) err = observer.Close() require.NoError(t, err) } + +func mockSQLForPassCreateConnCheck(mock sqlmock.Sqlmock) { + mock.ExpectQuery("select character_set_name from information_schema.character_sets " + + "where character_set_name = 'gbk';").WillReturnRows( + sqlmock.NewRows([]string{"character_set_name"}).AddRow("gbk"), + ) + columns := []string{"Variable_name", "Value"} + mock.ExpectQuery("show session variables like 'allow_auto_random_explicit_insert';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("allow_auto_random_explicit_insert", "0"), + ) + mock.ExpectQuery("show session variables like 'tidb_txn_mode';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("tidb_txn_mode", "pessimistic"), + ) + mock.ExpectQuery("show session variables like 'transaction_isolation';").WillReturnRows( + sqlmock.NewRows(columns).AddRow("transaction_isolation", "REPEATED-READ"), + ) + mock.ExpectQuery("show session variables like 'tidb_placement_mode';"). + WillReturnRows( + sqlmock.NewRows(columns). + AddRow("tidb_placement_mode", "IGNORE"), + ) + mock.ExpectQuery("show session variables like 'tidb_enable_external_ts_read';"). + WillReturnRows( + sqlmock.NewRows(columns). + AddRow("tidb_enable_external_ts_read", "OFF"), + ) + mock.ExpectClose() + // mock.ExpectQuery("select tidb_version()"). + // WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) + // mock.ExpectQuery("select tidb_version()"). + // WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) + // mock.ExpectClose() +} From ec856f931195a09e788756fd974800f21ef437e8 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 20 Aug 2024 19:29:09 +0800 Subject: [PATCH 10/50] test(MySQLDBConnector): add unit tests for `MySQLDBConnector` --- cdc/sink/ddlsink/mysql/async_ddl.go | 2 +- pkg/sink/mysql/db_helper.go | 25 +-- pkg/sink/mysql/mock_db.go | 3 + pkg/sink/mysql/mysql_connector.go | 21 ++- pkg/sink/mysql/mysql_connector_test.go | 208 +++++++++++++++++++++++++ pkg/sink/observer/observer.go | 6 +- pkg/sink/observer/observer_test.go | 18 +-- pkg/sink/observer/tidb_test.go | 152 +++++++++--------- 8 files changed, 319 insertions(+), 116 deletions(-) create mode 100644 pkg/sink/mysql/mysql_connector_test.go diff --git a/cdc/sink/ddlsink/mysql/async_ddl.go b/cdc/sink/ddlsink/mysql/async_ddl.go index 1692a394f37..d5898551b29 100644 --- a/cdc/sink/ddlsink/mysql/async_ddl.go +++ b/cdc/sink/ddlsink/mysql/async_ddl.go @@ -152,7 +152,7 @@ func (m *DDLSink) doCheck(ctx context.Context, table model.TableName) (done bool return true } - ret := m.db.QueryRowContext(ctx, fmt.Sprintf(checkRunningAddIndexSQL, table.Schema, table.Table)) + ret := m.connector.CurrentDB.QueryRowContext(ctx, fmt.Sprintf(checkRunningAddIndexSQL, table.Schema, table.Table)) if ret.Err() != nil { log.Error("check async exec ddl failed", zap.String("namespace", m.id.Namespace), diff --git a/pkg/sink/mysql/db_helper.go b/pkg/sink/mysql/db_helper.go index 69329dcda84..1a515e621cb 100644 --- a/pkg/sink/mysql/db_helper.go +++ b/pkg/sink/mysql/db_helper.go @@ -32,32 +32,32 @@ import ( "go.uber.org/zap" ) -// GenerateDSNs generates the DSN with the given config. -// GenerateDSNs uses the provided dbConnFactory to create a temporary connection +// generateDSNs generates the DSNs with the given config. +// generateDSNs uses the provided dbConnFactory to create a temporary connection // to the downstream database specified by the sinkURI. This temporary connection // is used to query important information from the downstream database, such as // version, charset, and other relevant details. After the required information // is retrieved, the temporary connection is closed. The retrieved data is then // used to populate additional parameters into the Sink URI, refining -// the connection URL (dsnStr). -func GenerateDSNs(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory ConnectionFactory) ([]string, error) { - // dsn format of the driver: - // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] +// the connection URL. +func generateDSNs(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory ConnectionFactory) ([]string, error) { dsnCfgs, err := GetDSNCfgs(sinkURI, cfg) if err != nil { return nil, err } var oneOfErrs error - dsn := make([]string, len(dsnCfgs)) + dsnList := make([]string, len(dsnCfgs)) for i := 0; i < len(dsnCfgs); i++ { - dsn[i], err = checkAndGenerateDSNByConfig(ctx, dsnCfgs[i], cfg, dbConnFactory) + // dsn format of the driver: + // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] + dsnList[i], err = checkAndGenerateDSNByConfig(ctx, dsnCfgs[i], cfg, dbConnFactory) if err != nil { oneOfErrs = err } } - return dsn, oneOfErrs + return dsnList, oneOfErrs } func checkAndGenerateDSNByConfig(ctx context.Context, dsnCfg *dmysql.Config, cfg *Config, dbConnFactory ConnectionFactory) (string, error) { @@ -234,8 +234,11 @@ func GetTestDB(ctx context.Context, dbConfig *dmysql.Config, dbConnFactory Conne return testDB, err } -// TODO: update comments -// GetDSNCfgs generates basic DSNs from the given config. +// GetDSNCfgs generates DSN configurations for multiple addresses contained in the given sinkURI. +// If the sinkURI contains multiple host addresses, the function will generate a DSN configuration +// for each address. It uses the provided `Config` to set various parameters such as TLS, timeouts, +// and the database timezone. The function supports IPv6 address formats and ensures that appropriate +// query parameters are applied. The DSN configurations are returned as a slice of `*mysql.Config` objects. func GetDSNCfgs(sinkURI *url.URL, cfg *Config) ([]*dmysql.Config, error) { // dsn format of the driver: // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] diff --git a/pkg/sink/mysql/mock_db.go b/pkg/sink/mysql/mock_db.go index d54bf15848a..f16fac757e6 100644 --- a/pkg/sink/mysql/mock_db.go +++ b/pkg/sink/mysql/mock_db.go @@ -55,6 +55,9 @@ func MockTestDB() (*sql.DB, error) { WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) mock.ExpectQuery("select tidb_version()"). WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) + + // Once mock.ExpectClose() has been called, the sql.DB can be closed multiple times + // without raising any errors. mock.ExpectClose() return db, nil } diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go index 162889c3185..8f442f7271b 100644 --- a/pkg/sink/mysql/mysql_connector.go +++ b/pkg/sink/mysql/mysql_connector.go @@ -27,20 +27,19 @@ type MySQLDBConnector struct { nextTry int // dbConnFactory is a Factory function for MySQLDBConnector that creates database connections. - dbConnFactory Factory + dbConnFactory IDBConnectionFactory // configureDBWhenSwitch stores the function that will be automatically invoked // to configure a new MySQL connection after switching to it using the // SwitchToAvailableMySQLDB function. This function will be set by the // ConfigureDBWhenSwitch method and can be applied to both newly established // connections and the current active connection if required. - configureDBWhenSwitch ConfigureDBWhenSwitch + configureDBWhenSwitch func() } -type ConfigureDBWhenSwitch func() - +// New a MySQLDBConnector for creating DB connection. func NewMySQLDBConnector(ctx context.Context, cfg *Config, sinkURI *url.URL) (*MySQLDBConnector, error) { - return NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, CreateMySQLDBConn) + return NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, &DBConnectionFactory{}) } // New a MySQLDBConnector by the given factory function for creating DB connection. @@ -48,17 +47,17 @@ func NewMySQLDBConnector(ctx context.Context, cfg *Config, sinkURI *url.URL) (*M // [scheme]://[user[:password]@][host[:port]][,host[:port]][,host[:port]][/path][?param1=value1¶mN=valueN] // User must ensure that each address ([host[:port]]) in the sinkURI (if there are multiple addresses) // is valid, otherwise returns an error. -func NewMySQLDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *url.URL, dbConnFactory Factory) (*MySQLDBConnector, error) { +func NewMySQLDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *url.URL, dbConnFactory IDBConnectionFactory) (*MySQLDBConnector, error) { if dbConnFactory == nil { - dbConnFactory = CreateMySQLDBConn + dbConnFactory = &DBConnectionFactory{} } - // GenerateDSN function parses multiple addresses from the URL (if any) + // generateDSNs function parses multiple addresses from the URL (if any) // and generates a DSN (Data Source Name) for each one. // For each DSN, the function attempts to create a connection and perform a Ping // to verify its availability. // If any of the DSNs are unavailable, the function immediately returns an error. - dsnList, err := GenerateDSNs(ctx, sinkURI, cfg, dbConnFactory) + dsnList, err := generateDSNs(ctx, sinkURI, cfg, dbConnFactory.CreateTemporaryConnection) if err != nil { return nil, err } @@ -104,7 +103,7 @@ func (c *MySQLDBConnector) SwitchToAvailableMySQLDB(ctx context.Context) error { continue } - db, err := c.dbConnFactory(ctx, c.dsnList[c.nextTry]) + db, err := c.dbConnFactory.CreateStandardConnection(ctx, c.dsnList[c.nextTry]) c.nextTry = (c.nextTry + 1) % len(c.dsnList) if err == nil { c.CurrentDB = db @@ -125,7 +124,7 @@ func (c *MySQLDBConnector) SwitchToAvailableMySQLDB(ctx context.Context) error { // MySQLDBConnector.CurrentDB also requires configuration, you can set // `needConfigureCurrentDB` to true, and this function will automatically apply // the configuration function `f` to it as well. -func (c *MySQLDBConnector) ConfigureDBWhenSwitch(f ConfigureDBWhenSwitch, needConfigureCurrentDB bool) { +func (c *MySQLDBConnector) ConfigureDBWhenSwitch(f func(), needConfigureCurrentDB bool) { if f == nil { return } diff --git a/pkg/sink/mysql/mysql_connector_test.go b/pkg/sink/mysql/mysql_connector_test.go new file mode 100644 index 00000000000..865fb1bf4c7 --- /dev/null +++ b/pkg/sink/mysql/mysql_connector_test.go @@ -0,0 +1,208 @@ +package mysql + +import ( + "context" + "database/sql" + "errors" + "net/url" + "testing" + + "github.com/DATA-DOG/go-sqlmock" + "github.com/stretchr/testify/require" +) + +// Test NewMySQLDBConnectorWithFactory for successful single address initialization +func TestNewMySQLDBConnectorWithFactory_SingleAddressSuccess(t *testing.T) { + ctx := context.Background() + sinkURI, _ := url.Parse("mysql://user:password@localhost") + cfg := NewConfig() + + dbConnFactory := NewDBConnectionFactoryForTest() + dbConnFactory.SetStandardConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { + db, mock, err := sqlmock.New( + sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual), + sqlmock.MonitorPingsOption(true)) + require.Nil(t, err) + mock.ExpectPing() + mock.ExpectClose() + return db, nil + }) + + connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + require.NoError(t, err) + require.NotNil(t, connector) + require.NotNil(t, connector.CurrentDB) + require.NoError(t, connector.CurrentDB.Ping()) + require.NoError(t, connector.CurrentDB.Close()) +} + +// Test NewMySQLDBConnectorWithFactory for successful multiple addresses initialization +func TestNewMySQLDBConnectorWithFactory_MultiAddressSuccess(t *testing.T) { + ctx := context.Background() + // Create a sinkURI which contains 3 addresses + sinkURI, _ := url.Parse("mysql://user:password@localhost,localhost,localhost") + cfg := NewConfig() + + dbConnFactory := &DBConnectionFactoryForTest{} + numCallTemporary := 0 + dbConnFactory.SetTemporaryConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { + numCallTemporary++ + db, err := MockTestDB() + require.NoError(t, err) + return db, nil + }) + numCallStandard := 0 + dbConnFactory.SetStandardConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { + numCallStandard++ + db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) + require.Nil(t, err) + mock.ExpectClose() + return db, nil + }) + + connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + require.NoError(t, err) + require.NotNil(t, connector) + require.NotNil(t, connector.CurrentDB) + require.Equal(t, numCallStandard, 1) + require.Equal(t, numCallTemporary, 3) + require.NoError(t, connector.CurrentDB.Close()) +} + +// Test NewMySQLDBConnectorWithFactory for error when DSN fail +func TestNewMySQLDBConnectorWithFactory_generateDSNsFail(t *testing.T) { + ctx := context.Background() + sinkURI, _ := url.Parse("mysql://user:password@localhost") + cfg := NewConfig() + + dbConnFactory := &DBConnectionFactoryForTest{} + dbConnFactory.SetTemporaryConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { + err := errors.New("") + require.Error(t, err) + return nil, err + }) + numCallStandard := 0 + dbConnFactory.SetStandardConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { + numCallStandard++ + return nil, nil + }) + + connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + require.Error(t, err) + require.Nil(t, connector) + require.Equal(t, numCallStandard, 0) +} + +// Test SwitchToAvailableMySQLDB when current DB is valid +func TestSwitchToAvailableMySQLDB_CurrentDBValid(t *testing.T) { + ctx := context.Background() + sinkURI, _ := url.Parse("mysql://user:password@localhost,localhost,localhost") + cfg := NewConfig() + + dbConnFactory := NewDBConnectionFactoryForTest() + dbConnFactory.SetStandardConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { + db, mock, err := sqlmock.New( + sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual), + sqlmock.MonitorPingsOption(true)) + require.Nil(t, err) + mock.ExpectPing() + mock.ExpectClose() + return db, nil + }) + + connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + require.NoError(t, err) + require.NotNil(t, connector) + require.NotNil(t, connector.CurrentDB) + dbBeforeSwitch := connector.CurrentDB + + require.NoError(t, connector.SwitchToAvailableMySQLDB(ctx)) + dbAfterSwitch := connector.CurrentDB + require.Equal(t, dbBeforeSwitch, dbAfterSwitch) + require.NoError(t, connector.CurrentDB.Close()) +} + +// Test SwitchToAvailableMySQLDB when current DB is invalid and switches to a new DB +func TestSwitchToAvailableMySQLDB_SwitchDB(t *testing.T) { + ctx := context.Background() + sinkURI, _ := url.Parse("mysql://user:password@localhost:123,localhost:456,localhost:789") + cfg := NewConfig() + + mockDB := func() (*sql.DB, error) { + db, mock, err := sqlmock.New( + sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual), + sqlmock.MonitorPingsOption(true)) + require.Nil(t, err) + mock.ExpectPing() + mock.ExpectClose() + return db, nil + } + + var dbCandidates [3]*sql.DB + for i := 0; i < len(dbCandidates); i++ { + dbCandidates[i], _ = mockDB() + } + + candidatesIndex := 0 + dbConnFactory := NewDBConnectionFactoryForTest() + dbConnFactory.SetStandardConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { + db := dbCandidates[candidatesIndex] + candidatesIndex++ + return db, nil + }) + + connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + require.NoError(t, err) + require.NotNil(t, connector) + require.NotNil(t, connector.CurrentDB) + + require.NoError(t, connector.CurrentDB.Ping()) + for i := 0; i < len(dbCandidates); i++ { + dbBeforeSwitch := connector.CurrentDB + require.NoError(t, dbCandidates[i].Close()) + require.Error(t, dbCandidates[i].Ping()) + if i != len(dbCandidates)-1 { + require.NoError(t, connector.SwitchToAvailableMySQLDB(ctx)) + require.NoError(t, connector.CurrentDB.Ping()) + dbAfterSwitch := connector.CurrentDB + require.NotEqual(t, dbBeforeSwitch, dbAfterSwitch) + } + } + + require.NoError(t, connector.CurrentDB.Close()) +} + +// Test ConfigureDBWhenSwitch to apply configuration function +func TestConfigureDBWhenSwitch(t *testing.T) { + ctx := context.Background() + sinkURI, _ := url.Parse("mysql://user:password@localhost,localhost") + cfg := NewConfig() + + dbConnFactory := NewDBConnectionFactoryForTest() + dbConnFactory.SetStandardConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { + db, mock, err := sqlmock.New( + sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual), + sqlmock.MonitorPingsOption(true)) + require.Nil(t, err) + mock.ExpectPing() + mock.ExpectClose() + return db, nil + }) + + connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + require.NoError(t, err) + require.NotNil(t, connector) + require.NotNil(t, connector.CurrentDB) + + numCallConfigure := 0 + connector.ConfigureDBWhenSwitch(func() { + numCallConfigure++ + }, true) + + require.NoError(t, connector.CurrentDB.Ping()) + require.NoError(t, connector.CurrentDB.Close()) + require.NoError(t, connector.SwitchToAvailableMySQLDB(ctx)) + require.NoError(t, connector.CurrentDB.Ping()) + require.NoError(t, connector.CurrentDB.Close()) + require.Equal(t, numCallConfigure, 2) +} diff --git a/pkg/sink/observer/observer.go b/pkg/sink/observer/observer.go index 8cf709a6b9c..fc0e505802f 100644 --- a/pkg/sink/observer/observer.go +++ b/pkg/sink/observer/observer.go @@ -37,14 +37,14 @@ type Observer interface { // NewObserverOpt represents available options when creating a new observer. type NewObserverOpt struct { - dbConnFactory pmysql.ConnectionFactory + dbConnFactory pmysql.IDBConnectionFactory } // NewObserverOption configures NewObserverOpt. type NewObserverOption func(*NewObserverOpt) // WithDBConnFactory specifies factory to create db connection. -func WithDBConnFactory(factory pmysql.ConnectionFactory) NewObserverOption { +func WithDBConnFactory(factory pmysql.IDBConnectionFactory) NewObserverOption { return func(opt *NewObserverOpt) { opt.dbConnFactory = factory } @@ -59,7 +59,7 @@ func NewObserver( opts ...NewObserverOption, ) (Observer, error) { creator := func() (Observer, error) { - options := &NewObserverOpt{dbConnFactory: pmysql.CreateMySQLDBConn} + options := &NewObserverOpt{dbConnFactory: &pmysql.DBConnectionFactory{}} for _, opt := range opts { opt(options) } diff --git a/pkg/sink/observer/observer_test.go b/pkg/sink/observer/observer_test.go index 225218f2ec2..2aed0b6c036 100644 --- a/pkg/sink/observer/observer_test.go +++ b/pkg/sink/observer/observer_test.go @@ -39,29 +39,19 @@ func newTestMockDB(t *testing.T) (db *sql.DB, mock sqlmock.Sqlmock) { func TestNewObserver(t *testing.T) { t.Parallel() - dbIndex := 0 - mockGetDBConn := func(ctx context.Context, dsnStr string) (*sql.DB, error) { - defer func() { dbIndex++ }() - - if dbIndex == 0 { - // test db - db, err := pmysql.MockTestDB() - require.Nil(t, err) - return db, nil - } - - // normal db + dbConnFactory := pmysql.NewDBConnectionFactoryForTest() + dbConnFactory.SetStandardConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { db, mock := newTestMockDB(t) mock.ExpectBegin() mock.ExpectClose() return db, nil - } + }) ctx := context.Background() sinkURI := "mysql://127.0.0.1:21347/" obs, err := NewObserver(ctx, model.DefaultChangeFeedID("test"), sinkURI, config.GetDefaultReplicaConfig(), - WithDBConnFactory(mockGetDBConn)) + WithDBConnFactory(dbConnFactory)) require.NoError(t, err) for i := 0; i < 10; i++ { err = obs.Tick(ctx) diff --git a/pkg/sink/observer/tidb_test.go b/pkg/sink/observer/tidb_test.go index 8a113fa4c95..1e4b673bec1 100644 --- a/pkg/sink/observer/tidb_test.go +++ b/pkg/sink/observer/tidb_test.go @@ -27,60 +27,60 @@ import ( func TestTiDBObserver(t *testing.T) { t.Parallel() - db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) - require.NoError(t, err) - - mockSQLForPassCreateConnCheck(mock) - mock.ExpectQuery(queryConnIdleDurationStmt). - WillReturnRows( - sqlmock.NewRows( - []string{"time", "instance", "in_txn", "quantile", "value"}). - AddRow("2023-01-16 17:22:16.881000", "10.2.6.127:11080", 0, 0.9, 0.309), - ) + dbConnFactory := mysql.NewDBConnectionFactoryForTest() + dbConnFactory.SetStandardConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { + db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) + require.NoError(t, err) - mock.ExpectQuery(queryConnCountStmt). - WillReturnRows( - sqlmock.NewRows( - []string{"time", "instance", "value"}). - AddRow("2023-01-10 16:44:39.123000", "10.2.6.127:11080", 24), - ) + mock.ExpectQuery(queryConnIdleDurationStmt). + WillReturnRows( + sqlmock.NewRows( + []string{"time", "instance", "in_txn", "quantile", "value"}). + AddRow("2023-01-16 17:22:16.881000", "10.2.6.127:11080", 0, 0.9, 0.309), + ) - mock.ExpectQuery(queryQueryDurationStmt). - WillReturnRows( - sqlmock.NewRows( - []string{"time", "instance", "sql_type", "value"}). - AddRow("2023-01-10 16:47:08.283000", "10.2.6.127:11080", - "Begin", 0.0018886375591793793). - AddRow("2023-01-10 16:47:08.283000", "10.2.6.127:11080", - "Insert", 0.014228768066070199). - AddRow("2023-01-10 16:47:08.283000", "10.2.6.127:11080", - "Delete", nil). - AddRow("2023-01-10 16:47:08.283000", "10.2.6.127:11080", - "Commit", 0.0004933262664880737), - ) + mock.ExpectQuery(queryConnCountStmt). + WillReturnRows( + sqlmock.NewRows( + []string{"time", "instance", "value"}). + AddRow("2023-01-10 16:44:39.123000", "10.2.6.127:11080", 24), + ) - mock.ExpectQuery(queryTxnDurationStmt). - WillReturnRows( - sqlmock.NewRows( - []string{"time", "instance", "type", "value"}). - AddRow("2023-01-10 16:50:38.153000", "10.2.6.127:11080", - "abort", nil). - AddRow("2023-01-10 16:50:38.153000", "10.2.6.127:11080", - "commit", 0.06155323076923076). - AddRow("2023-01-10 16:50:38.153000", "10.2.6.127:11080", - "rollback", nil), - ) - mock.ExpectClose() + mock.ExpectQuery(queryQueryDurationStmt). + WillReturnRows( + sqlmock.NewRows( + []string{"time", "instance", "sql_type", "value"}). + AddRow("2023-01-10 16:47:08.283000", "10.2.6.127:11080", + "Begin", 0.0018886375591793793). + AddRow("2023-01-10 16:47:08.283000", "10.2.6.127:11080", + "Insert", 0.014228768066070199). + AddRow("2023-01-10 16:47:08.283000", "10.2.6.127:11080", + "Delete", nil). + AddRow("2023-01-10 16:47:08.283000", "10.2.6.127:11080", + "Commit", 0.0004933262664880737), + ) - mockGetDBConnImpl := func(ctx context.Context, dsnStr string) (*sql.DB, error) { + mock.ExpectQuery(queryTxnDurationStmt). + WillReturnRows( + sqlmock.NewRows( + []string{"time", "instance", "type", "value"}). + AddRow("2023-01-10 16:50:38.153000", "10.2.6.127:11080", + "abort", nil). + AddRow("2023-01-10 16:50:38.153000", "10.2.6.127:11080", + "commit", 0.06155323076923076). + AddRow("2023-01-10 16:50:38.153000", "10.2.6.127:11080", + "rollback", nil), + ) + mock.ExpectClose() return db, nil - } + }) + cfg := mysql.NewConfig() sinkURIStr := "mysql://127.0.0.1:3306/" sinkURI, err := url.Parse(sinkURIStr) require.NoError(t, err) ctx := context.Background() - connector, err := mysql.NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, mockGetDBConnImpl) + connector, err := mysql.NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) require.NoError(t, err) observer := NewTiDBObserver(connector) @@ -90,35 +90,35 @@ func TestTiDBObserver(t *testing.T) { require.NoError(t, err) } -func mockSQLForPassCreateConnCheck(mock sqlmock.Sqlmock) { - mock.ExpectQuery("select character_set_name from information_schema.character_sets " + - "where character_set_name = 'gbk';").WillReturnRows( - sqlmock.NewRows([]string{"character_set_name"}).AddRow("gbk"), - ) - columns := []string{"Variable_name", "Value"} - mock.ExpectQuery("show session variables like 'allow_auto_random_explicit_insert';").WillReturnRows( - sqlmock.NewRows(columns).AddRow("allow_auto_random_explicit_insert", "0"), - ) - mock.ExpectQuery("show session variables like 'tidb_txn_mode';").WillReturnRows( - sqlmock.NewRows(columns).AddRow("tidb_txn_mode", "pessimistic"), - ) - mock.ExpectQuery("show session variables like 'transaction_isolation';").WillReturnRows( - sqlmock.NewRows(columns).AddRow("transaction_isolation", "REPEATED-READ"), - ) - mock.ExpectQuery("show session variables like 'tidb_placement_mode';"). - WillReturnRows( - sqlmock.NewRows(columns). - AddRow("tidb_placement_mode", "IGNORE"), - ) - mock.ExpectQuery("show session variables like 'tidb_enable_external_ts_read';"). - WillReturnRows( - sqlmock.NewRows(columns). - AddRow("tidb_enable_external_ts_read", "OFF"), - ) - mock.ExpectClose() - // mock.ExpectQuery("select tidb_version()"). - // WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) - // mock.ExpectQuery("select tidb_version()"). - // WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) - // mock.ExpectClose() -} +// func mockSQLForPassCreateConnCheck(mock sqlmock.Sqlmock) { +// mock.ExpectQuery("select character_set_name from information_schema.character_sets " + +// "where character_set_name = 'gbk';").WillReturnRows( +// sqlmock.NewRows([]string{"character_set_name"}).AddRow("gbk"), +// ) +// columns := []string{"Variable_name", "Value"} +// mock.ExpectQuery("show session variables like 'allow_auto_random_explicit_insert';").WillReturnRows( +// sqlmock.NewRows(columns).AddRow("allow_auto_random_explicit_insert", "0"), +// ) +// mock.ExpectQuery("show session variables like 'tidb_txn_mode';").WillReturnRows( +// sqlmock.NewRows(columns).AddRow("tidb_txn_mode", "pessimistic"), +// ) +// mock.ExpectQuery("show session variables like 'transaction_isolation';").WillReturnRows( +// sqlmock.NewRows(columns).AddRow("transaction_isolation", "REPEATED-READ"), +// ) +// mock.ExpectQuery("show session variables like 'tidb_placement_mode';"). +// WillReturnRows( +// sqlmock.NewRows(columns). +// AddRow("tidb_placement_mode", "IGNORE"), +// ) +// mock.ExpectQuery("show session variables like 'tidb_enable_external_ts_read';"). +// WillReturnRows( +// sqlmock.NewRows(columns). +// AddRow("tidb_enable_external_ts_read", "OFF"), +// ) +// mock.ExpectClose() +// // mock.ExpectQuery("select tidb_version()"). +// // WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) +// // mock.ExpectQuery("select tidb_version()"). +// // WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) +// // mock.ExpectClose() +// } From 6a797aa37e7ced376bd3d3e347e2c75713c0f949 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 21 Aug 2024 15:17:42 +0800 Subject: [PATCH 11/50] fix(TestGenerateDSNByConfig): fix missed `character_set_name` query --- pkg/sink/mysql/config_test.go | 8 ++++++++ pkg/sink/mysql/db_helper.go | 24 ++++++++++++------------ pkg/sink/mysql/mock_db.go | 8 ++++---- 3 files changed, 24 insertions(+), 16 deletions(-) diff --git a/pkg/sink/mysql/config_test.go b/pkg/sink/mysql/config_test.go index 0e7b35fa57e..8810817753b 100644 --- a/pkg/sink/mysql/config_test.go +++ b/pkg/sink/mysql/config_test.go @@ -133,6 +133,10 @@ func TestGenerateDSNByConfig(t *testing.T) { sqlmock.NewRows(columns). AddRow("tidb_enable_external_ts_read", "OFF"), ) + mock.ExpectQuery("select character_set_name from information_schema.character_sets " + + "where character_set_name = 'gbk';").WillReturnRows( + sqlmock.NewRows([]string{"character_set_name"}).AddRow("gbk"), + ) dsnStr, err = generateDSNByConfig(context.TODO(), dsn, cfg, db) require.Nil(t, err) expectedCfg := []string{ @@ -162,6 +166,10 @@ func TestGenerateDSNByConfig(t *testing.T) { sqlmock.NewRows(columns). AddRow("tidb_enable_external_ts_read", "OFF"), ) + mock.ExpectQuery("select character_set_name from information_schema.character_sets " + + "where character_set_name = 'gbk';").WillReturnRows( + sqlmock.NewRows([]string{"character_set_name"}).AddRow("gbk"), + ) dsnStr, err = generateDSNByConfig(context.TODO(), dsn, cfg, db) require.Nil(t, err) expectedCfg = []string{ diff --git a/pkg/sink/mysql/db_helper.go b/pkg/sink/mysql/db_helper.go index 1a515e621cb..4ad04df1855 100644 --- a/pkg/sink/mysql/db_helper.go +++ b/pkg/sink/mysql/db_helper.go @@ -67,18 +67,6 @@ func checkAndGenerateDSNByConfig(ctx context.Context, dsnCfg *dmysql.Config, cfg } defer testDB.Close() - // check if GBK charset is supported by downstream - var gbkSupported bool - gbkSupported, err = checkCharsetSupport(ctx, testDB, charset.CharsetGBK) - if err != nil { - return "", err - } - if !gbkSupported { - log.Warn("GBK charset is not supported by the downstream. "+ - "Some types of DDLs may fail to execute", - zap.String("host", dsnCfg.Addr)) - } - dsnStr, err := generateDSNByConfig(ctx, dsnCfg, cfg, testDB) if err != nil { return "", err @@ -168,6 +156,18 @@ func generateDSNByConfig( // set the `tidb_enable_external_ts_read` to `OFF`, so cdc could write to the sink dsnCfg.Params["tidb_enable_external_ts_read"] = fmt.Sprintf(`"%s"`, tidbEnableExternalTSRead) } + // check if GBK charset is supported by downstream + var gbkSupported bool + gbkSupported, err = checkCharsetSupport(ctx, testDB, charset.CharsetGBK) + if err != nil { + return "", err + } + if !gbkSupported { + log.Warn("GBK charset is not supported by the downstream. "+ + "Some types of DDLs may fail to execute", + zap.String("host", dsnCfg.Addr)) + } + dsnClone := dsnCfg.Clone() dsnClone.Passwd = "******" log.Info("sink uri is configured", zap.String("dsn", dsnClone.FormatDSN())) diff --git a/pkg/sink/mysql/mock_db.go b/pkg/sink/mysql/mock_db.go index f16fac757e6..571e94bf9a1 100644 --- a/pkg/sink/mysql/mock_db.go +++ b/pkg/sink/mysql/mock_db.go @@ -27,10 +27,6 @@ func MockTestDB() (*sql.DB, error) { return nil, err } - mock.ExpectQuery("select character_set_name from information_schema.character_sets " + - "where character_set_name = 'gbk';").WillReturnRows( - sqlmock.NewRows([]string{"character_set_name"}).AddRow("gbk"), - ) columns := []string{"Variable_name", "Value"} mock.ExpectQuery("show session variables like 'allow_auto_random_explicit_insert';").WillReturnRows( sqlmock.NewRows(columns).AddRow("allow_auto_random_explicit_insert", "0"), @@ -51,6 +47,10 @@ func MockTestDB() (*sql.DB, error) { sqlmock.NewRows(columns). AddRow("tidb_enable_external_ts_read", "OFF"), ) + mock.ExpectQuery("select character_set_name from information_schema.character_sets " + + "where character_set_name = 'gbk';").WillReturnRows( + sqlmock.NewRows([]string{"character_set_name"}).AddRow("gbk"), + ) mock.ExpectQuery("select tidb_version()"). WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}).AddRow("5.7.25-TiDB-v4.0.0-beta-191-ga1b3e3b")) mock.ExpectQuery("select tidb_version()"). From 93aee29880f5b491fae406745cef88a15228bb88 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 21 Aug 2024 15:32:00 +0800 Subject: [PATCH 12/50] fix(TestNewMySQLBackend): fix the idempotency of close mysql connector --- cdc/sink/dmlsink/txn/mysql/mysql.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index b66623f9700..f03c9cf79ec 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -260,7 +260,7 @@ func (s *mysqlBackend) Close() (err error) { } if s.connector.CurrentDB != nil { err = s.connector.CurrentDB.Close() - s.connector = nil + s.connector.CurrentDB = nil } return } From d0c844a3ac8719193d9f5e71c12e60da61160863 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 21 Aug 2024 16:58:08 +0800 Subject: [PATCH 13/50] test(mysql connector): add TestConfigureDBWhenSwitch_NilConfigureFunction --- pkg/sink/mysql/mysql_connector_test.go | 32 ++++++++++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/pkg/sink/mysql/mysql_connector_test.go b/pkg/sink/mysql/mysql_connector_test.go index 865fb1bf4c7..ea8043fdc49 100644 --- a/pkg/sink/mysql/mysql_connector_test.go +++ b/pkg/sink/mysql/mysql_connector_test.go @@ -69,7 +69,7 @@ func TestNewMySQLDBConnectorWithFactory_MultiAddressSuccess(t *testing.T) { require.NoError(t, connector.CurrentDB.Close()) } -// Test NewMySQLDBConnectorWithFactory for error when DSN fail +// Test NewMySQLDBConnectorWithFactory for error when generate DSNs fail func TestNewMySQLDBConnectorWithFactory_generateDSNsFail(t *testing.T) { ctx := context.Background() sinkURI, _ := url.Parse("mysql://user:password@localhost") @@ -173,7 +173,7 @@ func TestSwitchToAvailableMySQLDB_SwitchDB(t *testing.T) { } // Test ConfigureDBWhenSwitch to apply configuration function -func TestConfigureDBWhenSwitch(t *testing.T) { +func TestConfigureDBWhenSwitch_Success(t *testing.T) { ctx := context.Background() sinkURI, _ := url.Parse("mysql://user:password@localhost,localhost") cfg := NewConfig() @@ -206,3 +206,31 @@ func TestConfigureDBWhenSwitch(t *testing.T) { require.NoError(t, connector.CurrentDB.Close()) require.Equal(t, numCallConfigure, 2) } + +// Test ConfigureDBWhenSwitch when configuration function is nil and switches to a new DB +func TestConfigureDBWhenSwitch_NilConfigureFunction(t *testing.T) { + ctx := context.Background() + sinkURI, _ := url.Parse("mysql://user:password@localhost,localhost") + cfg := NewConfig() + + dbConnFactory := NewDBConnectionFactoryForTest() + dbConnFactory.SetStandardConnectionFactory(func(ctx context.Context, dsnStr string) (*sql.DB, error) { + db, mock, err := sqlmock.New( + sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual), + sqlmock.MonitorPingsOption(true)) + require.Nil(t, err) + mock.ExpectClose() + return db, nil + }) + + connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + require.NoError(t, err) + require.NotNil(t, connector) + require.NotNil(t, connector.CurrentDB) + + connector.ConfigureDBWhenSwitch(nil, true) + + require.NoError(t, connector.CurrentDB.Close()) + require.NoError(t, connector.SwitchToAvailableMySQLDB(ctx)) + require.NoError(t, connector.CurrentDB.Close()) +} From 54addd58b8b5743a230da8d1bb07699884e9f27f Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 23 Aug 2024 15:54:28 +0800 Subject: [PATCH 14/50] feat(start_tidb_cluster): support launching multiple downstream TiDB instances within a single cluster --- .../_utils/start_tidb_cluster | 5 ++ .../_utils/start_tidb_cluster_impl | 49 +++++++++++-------- .../_utils/stop_tidb_cluster | 2 +- tests/integration_tests/_utils/test_prepare | 12 +++-- tests/integration_tests/api_v2/run.sh | 2 +- tests/integration_tests/autorandom/run.sh | 2 +- .../integration_tests/availability/capture.sh | 8 +-- tests/integration_tests/availability/run.sh | 2 +- tests/integration_tests/avro_basic/run.sh | 2 +- tests/integration_tests/bank/run.sh | 6 +-- .../integration_tests/batch_add_table/run.sh | 8 +-- .../batch_update_to_no_batch/run.sh | 8 +-- tests/integration_tests/bdr_mode/run.sh | 6 +-- .../canal_json_adapter_compatibility/run.sh | 4 +- .../integration_tests/canal_json_basic/run.sh | 4 +- .../canal_json_claim_check/run.sh | 2 +- .../canal_json_content_compatible/run.sh | 4 +- .../canal_json_handle_key_only/run.sh | 2 +- .../canal_json_storage_partition_table/run.sh | 10 ++-- .../capture_session_done_during_task/run.sh | 6 +-- .../run.sh | 2 +- .../changefeed_auto_stop/run.sh | 2 +- .../changefeed_dup_error_restart/run.sh | 4 +- .../integration_tests/changefeed_error/run.sh | 2 +- .../changefeed_pause_resume/run.sh | 2 +- .../changefeed_reconstruct/run.sh | 4 +- .../run.sh | 12 ++--- tests/integration_tests/charset_gbk/run.sh | 8 +-- .../ci_collation_compatibility/run.sh | 4 +- .../cli_tls_with_auth/run.sh | 4 +- tests/integration_tests/cli_with_auth/run.sh | 4 +- .../integration_tests/clustered_index/run.sh | 6 +-- tests/integration_tests/common_1/run.sh | 8 +-- .../consistent_partition_table/run.sh | 8 +-- .../consistent_replicate_ddl/run.sh | 14 +++--- .../consistent_replicate_gbk/run.sh | 4 +- .../consistent_replicate_nfs/run.sh | 4 +- .../consistent_replicate_storage_file/run.sh | 10 ++-- .../run.sh | 10 ++-- .../consistent_replicate_storage_s3/run.sh | 10 ++-- .../csv_storage_multi_tables_ddl/run.sh | 10 ++-- .../csv_storage_partition_table/run.sh | 10 ++-- .../csv_storage_update_pk_clustered/run.sh | 2 +- .../csv_storage_update_pk_nonclustered/run.sh | 2 +- tests/integration_tests/ddl_attributes/run.sh | 18 +++---- tests/integration_tests/ddl_manager/run.sh | 2 +- .../ddl_only_block_related_table/run.sh | 2 +- tests/integration_tests/ddl_puller_lag/run.sh | 4 +- tests/integration_tests/ddl_reentrant/run.sh | 4 +- tests/integration_tests/ddl_sequence/run.sh | 2 +- tests/integration_tests/default_value/run.sh | 8 +-- .../integration_tests/drop_many_tables/run.sh | 2 +- tests/integration_tests/event_filter/run.sh | 26 +++++----- .../force_replicate_table/run.sh | 4 +- tests/integration_tests/foreign_key/run.sh | 2 +- .../integration_tests/generate_column/run.sh | 4 +- tests/integration_tests/http_api/run.sh | 2 +- tests/integration_tests/http_api_tls/run.sh | 2 +- .../http_api_tls_with_user_auth/run.sh | 2 +- .../kafka_big_messages/run.sh | 2 +- .../kafka_big_messages_v2/run.sh | 2 +- .../kafka_column_selector/run.sh | 4 +- .../kafka_column_selector_avro/run.sh | 4 +- .../kafka_compression/run.sh | 2 +- tests/integration_tests/kafka_messages/run.sh | 24 ++++----- .../kafka_simple_basic/run.sh | 6 +-- .../kafka_simple_basic_avro/run.sh | 4 +- .../kafka_simple_claim_check/run.sh | 2 +- .../kafka_simple_claim_check_avro/run.sh | 2 +- .../kafka_simple_handle_key_only/run.sh | 2 +- .../kafka_simple_handle_key_only_avro/run.sh | 2 +- .../kafka_sink_error_resume/run.sh | 4 +- .../kill_owner_with_ddl/run.sh | 4 +- tests/integration_tests/many_pk_or_uk/run.sh | 2 +- tests/integration_tests/move_table/run.sh | 6 +-- .../mq_sink_dispatcher/run.sh | 2 +- tests/integration_tests/multi_capture/run.sh | 2 +- .../multi_cdc_cluster/run.sh | 4 +- tests/integration_tests/multi_rocks/run.sh | 2 +- tests/integration_tests/multi_source/run.sh | 28 +++++------ .../integration_tests/multi_tables_ddl/run.sh | 10 ++-- .../multi_tables_ddl_v2/run.sh | 10 ++-- tests/integration_tests/multi_topics/run.sh | 8 +-- .../integration_tests/multi_topics_v2/run.sh | 8 +-- .../integration_tests/new_ci_collation/run.sh | 2 +- .../open_protocol_claim_check/run.sh | 2 +- .../open_protocol_handle_key_only/run.sh | 2 +- .../owner_remove_table_error/run.sh | 2 +- tests/integration_tests/owner_resign/run.sh | 2 +- .../integration_tests/partition_table/run.sh | 10 ++-- .../processor_err_chan/run.sh | 4 +- .../processor_etcd_worker_delay/run.sh | 2 +- .../processor_resolved_ts_fallback/run.sh | 6 +-- .../processor_stop_delay/run.sh | 2 +- tests/integration_tests/region_merge/run.sh | 2 +- tests/integration_tests/resolve_lock/run.sh | 2 +- .../integration_tests/resourcecontrol/run.sh | 2 +- tests/integration_tests/row_format/run.sh | 2 +- tests/integration_tests/savepoint/run.sh | 2 +- tests/integration_tests/sequence/run.sh | 6 +-- .../server_config_compatibility/run.sh | 4 +- tests/integration_tests/simple/run.sh | 4 +- tests/integration_tests/sink_hang/run.sh | 4 +- tests/integration_tests/sink_retry/run.sh | 4 +- tests/integration_tests/split_region/run.sh | 4 +- tests/integration_tests/sql_mode/run.sh | 12 ++--- tests/integration_tests/synced_status/run.sh | 6 +-- .../synced_status_with_redo/run.sh | 6 +-- tests/integration_tests/syncpoint/run.sh | 14 +++--- .../syncpoint_check_ts/run.sh | 6 +-- .../integration_tests/tidb_mysql_test/run.sh | 2 +- tests/integration_tests/tiflash/run.sh | 2 +- 112 files changed, 329 insertions(+), 309 deletions(-) diff --git a/tests/integration_tests/_utils/start_tidb_cluster b/tests/integration_tests/_utils/start_tidb_cluster index 3113f7a2cbd..e7b214b7649 100755 --- a/tests/integration_tests/_utils/start_tidb_cluster +++ b/tests/integration_tests/_utils/start_tidb_cluster @@ -12,6 +12,7 @@ pd_config= tikv_config= retry_times=3 multiple_upstream_pd="false" +downstream_db=1 while [[ ${1} ]]; do case "${1}" in @@ -39,6 +40,10 @@ while [[ ${1} ]]; do multiple_upstream_pd=${2} shift ;; + --downstream_db) + downstream_db=${2} + shift + ;; *) echo "Unknown parameter: ${1}" >&2 exit 1 diff --git a/tests/integration_tests/_utils/start_tidb_cluster_impl b/tests/integration_tests/_utils/start_tidb_cluster_impl index 7c306179836..7370b83a515 100755 --- a/tests/integration_tests/_utils/start_tidb_cluster_impl +++ b/tests/integration_tests/_utils/start_tidb_cluster_impl @@ -11,6 +11,7 @@ tidb_config= pd_config= tikv_config= multiple_upstream_pd= +downstream_db= random_file_name= # Random generate the sockets config. @@ -48,6 +49,10 @@ while [[ ${1} ]]; do multiple_upstream_pd=${2} shift ;; + --downstream_db) + downstream_db=${2} + shift + ;; *) echo "Unknown parameter: ${1}" >&2 exit 1 @@ -245,14 +250,16 @@ tidb-server \ echo "Starting Downstream TiDB..." tidb-server -V -randomGenSocketsConf -tidb-server \ - -P ${DOWN_TIDB_PORT} \ - -config "$random_file_name" \ - --store tikv \ - --path ${DOWN_PD_HOST}:${DOWN_PD_PORT} \ - --status=${DOWN_TIDB_STATUS} \ - --log-file "$OUT_DIR/tidb_down.log" & +for i in $(seq 1 $downstream_db); do + randomGenSocketsConf + tidb-server \ + -P ${!DOWN_TIDB_PORT_${i}} \ + -config "$random_file_name" \ + --store tikv \ + --path ${DOWN_PD_HOST}:${DOWN_PD_PORT} \ + --status=${!DOWN_TIDB_STATUS_${i}} \ + --log-file "$OUT_DIR/tidb_down_$i.log" & +done echo "Verifying Upstream TiDB is started..." i=0 @@ -276,21 +283,23 @@ while ! mysql -uroot -h${UP_TIDB_HOST} -P${UP_TIDB_OTHER_PORT} --default-charact done echo "Verifying Downstream TiDB is started..." -i=0 -while ! mysql -uroot -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} --default-character-set utf8mb4 -e 'select * from mysql.tidb;'; do - i=$((i + 1)) - if [ "$i" -gt 60 ]; then - echo 'Failed to start downstream TiDB' - exit 1 - fi - sleep 2 +for index in $(seq 1 $downstream_db); do + i=0 + while ! mysql -uroot -h${DOWN_TIDB_HOST} -P${!DOWN_TIDB_PORT_${index}} --default-character-set utf8mb4 -e 'select * from mysql.tidb;'; do + i=$((i + 1)) + if [ "$i" -gt 60 ]; then + echo 'Failed to start downstream TiDB' + exit 1 + fi + sleep 2 + done done run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${UP_TIDB_HOST} ${UP_TIDB_PORT} -run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} -run_sql "CREATE user 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} -run_sql "GRANT select,insert,update,delete,index,create,drop,alter,create view,references ON *.* TO 'normal'@'%';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} -run_sql "FLUSH privileges" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} +run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} +run_sql "CREATE user 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} +run_sql "GRANT select,insert,update,delete,index,create,drop,alter,create view,references ON *.* TO 'normal'@'%';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} +run_sql "FLUSH privileges" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} cat - >"$OUT_DIR/tiflash-config.toml" </dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_TIDB_OTHER_PORT} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_TIDB_STATUS} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_TIDB_OTHER_STATUS} -t 2>/dev/null) &>/dev/null || true -kill -9 $(lsof -i tcp:${DOWN_TIDB_PORT} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${DOWN_TIDB_PORT_1} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${DOWN_TIDB_STATUS} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_PD_PORT_1} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_PD_PEER_PORT_1} -t 2>/dev/null) &>/dev/null || true diff --git a/tests/integration_tests/_utils/test_prepare b/tests/integration_tests/_utils/test_prepare index b08a0664864..1f2c897397e 100644 --- a/tests/integration_tests/_utils/test_prepare +++ b/tests/integration_tests/_utils/test_prepare @@ -4,10 +4,16 @@ UP_TIDB_OTHER_PORT=${UP_TIDB_OTHER_PORT:-4001} UP_TIDB_STATUS=${UP_TIDB_STATUS:-10080} UP_TIDB_OTHER_STATUS=${UP_TIDB_OTHER_STATUS:-10081} DOWN_TIDB_HOST=${DOWN_TIDB_HOST:-127.0.0.1} -DOWN_TIDB_PORT=${DOWN_TIDB_PORT:-3306} -DOWN_TIDB_STATUS=${DOWN_TIDB_STATUS:-20080} +; DOWN_TIDB_PORT=${DOWN_TIDB_PORT:-3306} +; DOWN_TIDB_STATUS=${DOWN_TIDB_STATUS:-20080} +DOWN_TIDB_PORT_1=${DOWN_TIDB_PORT_1:-3306} +DOWN_TIDB_STATUS_1=${DOWN_TIDB_STATUS_1:-20080} +DOWN_TIDB_PORT_2=${DOWN_TIDB_PORT_2:-3307} +DOWN_TIDB_STATUS_2=${DOWN_TIDB_STATUS_2:-20081} +DOWN_TIDB_PORT_3=${DOWN_TIDB_PORT_3:-3308} +DOWN_TIDB_STATUS_3=${DOWN_TIDB_STATUS_3:-20082} TLS_TIDB_HOST=${TLS_TIDB_HOST:-127.0.0.1} -TLS_TIDB_PORT=${TLS_TIDB_PORT:-3307} +TLS_TIDB_PORT=${TLS_TIDB_PORT:-3310} TLS_TIDB_STATUS=${TLS_TIDB_STATUS:-30080} UP_PD_HOST_1=${UP_PD_HOST_1:-127.0.0.1} diff --git a/tests/integration_tests/api_v2/run.sh b/tests/integration_tests/api_v2/run.sh index 865ee8ae66b..c794aac11c2 100644 --- a/tests/integration_tests/api_v2/run.sh +++ b/tests/integration_tests/api_v2/run.sh @@ -17,7 +17,7 @@ function prepare() { # create table to upstream. run_sql "CREATE DATABASE api_v2" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "CREATE DATABASE api_v2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "CREATE DATABASE api_v2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY cdc cli changefeed create -c="cf-blackhole" --sink-uri="blackhole://" diff --git a/tests/integration_tests/autorandom/run.sh b/tests/integration_tests/autorandom/run.sh index 8c69d8c0a0a..208a16c2ec4 100644 --- a/tests/integration_tests/autorandom/run.sh +++ b/tests/integration_tests/autorandom/run.sh @@ -35,7 +35,7 @@ function run() { esac run_sql_file $CUR/data/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists autorandom_test.table_a ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists autorandom_test.table_a ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY } diff --git a/tests/integration_tests/availability/capture.sh b/tests/integration_tests/availability/capture.sh index 0f538f24096..65f3f00b31a 100755 --- a/tests/integration_tests/availability/capture.sh +++ b/tests/integration_tests/availability/capture.sh @@ -15,7 +15,7 @@ function sql_check() { # check table availability. echo "run sql_check", ${DOWN_TIDB_HOST} - run_sql "SELECT id, val FROM test.availability1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "SELECT id, val FROM test.availability1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "id: 1" && check_contains "val: 1" && check_contains "id: 2" && @@ -30,13 +30,13 @@ function check_result() { function empty() { sql=$* - run_sql "$sql" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "$sql" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_not_contains "id:" } function nonempty() { sql=$* - run_sql "$sql" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "$sql" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "id:" } @@ -66,7 +66,7 @@ function test_kill_capture() { echo "owner id" $owner_id # wait for the tables to appear - check_table_exists test.availability1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 20 + check_table_exists test.availability1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 20 run_sql "INSERT INTO test.availability1(id, val) VALUES (1, 1);" ensure $MAX_RETRIES nonempty 'select id, val from test.availability1 where id=1 and val=1' diff --git a/tests/integration_tests/availability/run.sh b/tests/integration_tests/availability/run.sh index 1e1434125e4..a7a59475f53 100644 --- a/tests/integration_tests/availability/run.sh +++ b/tests/integration_tests/availability/run.sh @@ -12,7 +12,7 @@ CDC_BINARY=cdc.test SINK_TYPE=$1 export DOWN_TIDB_HOST -export DOWN_TIDB_PORT +export DOWN_TIDB_PORT_1 function prepare() { rm -rf $WORK_DIR && mkdir -p $WORK_DIR diff --git a/tests/integration_tests/avro_basic/run.sh b/tests/integration_tests/avro_basic/run.sh index 27f8b0b0d51..0a3977aef5a 100644 --- a/tests/integration_tests/avro_basic/run.sh +++ b/tests/integration_tests/avro_basic/run.sh @@ -52,7 +52,7 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/bank/run.sh b/tests/integration_tests/bank/run.sh index e0c1a5cd6de..10e5e85226b 100644 --- a/tests/integration_tests/bank/run.sh +++ b/tests/integration_tests/bank/run.sh @@ -17,11 +17,11 @@ function prepare() { # create table to upstream. run_sql "CREATE DATABASE bank" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "CREATE DATABASE bank" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "CREATE DATABASE bank" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - run_cdc_cli changefeed create --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT}/" + run_cdc_cli changefeed create --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1}/" } trap stop_tidb_cluster EXIT @@ -33,7 +33,7 @@ if [ "$SINK_TYPE" == "mysql" ]; then set -euxo pipefail GO111MODULE=on go run bank.go case.go -u "root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/bank" \ - -d "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT})/bank" --test-round=20000 \ + -d "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1})/bank" --test-round=20000 \ -a "${DOWN_TIDB_HOST}:${DOWN_TIDB_STATUS}" cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/batch_add_table/run.sh b/tests/integration_tests/batch_add_table/run.sh index 0989096627b..a02821ee0bc 100644 --- a/tests/integration_tests/batch_add_table/run.sh +++ b/tests/integration_tests/batch_add_table/run.sh @@ -49,15 +49,15 @@ function run_with_fast_create_table() { mysql -h ${UP_TIDB_HOST} -P ${UP_TIDB_PORT} -u root -D "test" -e "insert into t_$i values(1,2)" done - check_table_exists test.t_100 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists test.t_100 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql_file $CUR/data/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists batch_add_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists batch_add_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} # check the ddl of this table is skipped - check_table_not_exists test.t_1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_not_exists test.t_1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY @@ -95,7 +95,7 @@ function run_without_fast_create_table() { run_sql_file $CUR/data/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists batch_add_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists batch_add_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/batch_update_to_no_batch/run.sh b/tests/integration_tests/batch_update_to_no_batch/run.sh index 15a9b26f340..56ec26050cf 100644 --- a/tests/integration_tests/batch_update_to_no_batch/run.sh +++ b/tests/integration_tests/batch_update_to_no_batch/run.sh @@ -26,7 +26,7 @@ function run() { run_sql "set global tidb_enable_change_multi_schema = on" ${UP_TIDB_HOST} ${UP_TIDB_PORT} # This must be set before cdc server starts - run_sql "set global tidb_enable_change_multi_schema = on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set global tidb_enable_change_multi_schema = on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} # TiDB global variables cache 2 seconds at most sleep 2 @@ -60,9 +60,9 @@ function run() { run_sql_file $CUR/data/test_finish.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists batch_update_to_no_batch.v1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists batch_update_to_no_batch.recover_and_insert ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists batch_update_to_no_batch.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists batch_update_to_no_batch.v1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists batch_update_to_no_batch.recover_and_insert ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists batch_update_to_no_batch.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/bdr_mode/run.sh b/tests/integration_tests/bdr_mode/run.sh index 501ebaac7b8..0a7f106757e 100644 --- a/tests/integration_tests/bdr_mode/run.sh +++ b/tests/integration_tests/bdr_mode/run.sh @@ -37,16 +37,16 @@ function run() { run_cdc_cli changefeed create --sink-uri="$SINK_URI_2" -c "test-2" --server "http://127.0.0.1:8400" --config="$CUR/conf/down.toml" run_sql_file $CUR/data/up.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql_file $CUR/data/down.sql ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql_file $CUR/data/down.sql ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_sql_file $CUR/data/finished.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # syncpoint table should exists in secondary tidb, but does not exists in primary cluster - check_table_exists "tidb_cdc.syncpoint_v1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + check_table_exists "tidb_cdc.syncpoint_v1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 check_table_exists "tidb_cdc.syncpoint_v1" ${UP_TIDB_HOST} ${UP_TIDB_PORT} 60 check_table_exists "bdr_mode.finish_mark" ${UP_TIDB_HOST} ${UP_TIDB_PORT} 60 - check_table_exists "bdr_mode.finish_mark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + check_table_exists "bdr_mode.finish_mark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/canal_json_adapter_compatibility/run.sh b/tests/integration_tests/canal_json_adapter_compatibility/run.sh index f9c45af1dc3..0bfd803daaa 100644 --- a/tests/integration_tests/canal_json_adapter_compatibility/run.sh +++ b/tests/integration_tests/canal_json_adapter_compatibility/run.sh @@ -33,11 +33,11 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.binary_columns ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 600 + check_table_exists test.binary_columns ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 600 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 600 run_sql_file $CUR/data/data_gbk.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists test.binary_columns ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 600 + check_table_exists test.binary_columns ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 600 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 600 cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/canal_json_basic/run.sh b/tests/integration_tests/canal_json_basic/run.sh index 03b2c9dab73..c3c1b541d20 100644 --- a/tests/integration_tests/canal_json_basic/run.sh +++ b/tests/integration_tests/canal_json_basic/run.sh @@ -50,11 +50,11 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql_file $CUR/data/data_gbk.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/canal_json_claim_check/run.sh b/tests/integration_tests/canal_json_claim_check/run.sh index 1678931422e..cabd671f9de 100644 --- a/tests/integration_tests/canal_json_claim_check/run.sh +++ b/tests/integration_tests/canal_json_claim_check/run.sh @@ -36,7 +36,7 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/canal_json_content_compatible/run.sh b/tests/integration_tests/canal_json_content_compatible/run.sh index b8afab5d197..950bedf488d 100644 --- a/tests/integration_tests/canal_json_content_compatible/run.sh +++ b/tests/integration_tests/canal_json_content_compatible/run.sh @@ -50,11 +50,11 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql_file $CUR/data/data_gbk.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/canal_json_handle_key_only/run.sh b/tests/integration_tests/canal_json_handle_key_only/run.sh index 6c1cc18aef4..a337ebd661b 100644 --- a/tests/integration_tests/canal_json_handle_key_only/run.sh +++ b/tests/integration_tests/canal_json_handle_key_only/run.sh @@ -36,7 +36,7 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/canal_json_storage_partition_table/run.sh b/tests/integration_tests/canal_json_storage_partition_table/run.sh index be3cac83c86..b3283c25221 100644 --- a/tests/integration_tests/canal_json_storage_partition_table/run.sh +++ b/tests/integration_tests/canal_json_storage_partition_table/run.sh @@ -17,7 +17,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --logsuffix cdc0 run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8301" --logsuffix cdc1 @@ -32,10 +32,10 @@ function run() { sleep 8 # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists partition_table.t ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists partition_table.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - # check_table_exists partition_table.t2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists partition_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists partition_table.t ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists partition_table.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + # check_table_exists partition_table.t2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists partition_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/capture_session_done_during_task/run.sh b/tests/integration_tests/capture_session_done_during_task/run.sh index c22d70c8b5f..5466a096546 100644 --- a/tests/integration_tests/capture_session_done_during_task/run.sh +++ b/tests/integration_tests/capture_session_done_during_task/run.sh @@ -29,8 +29,8 @@ function run() { # will be task dispatched after changefeed starts. run_sql "CREATE DATABASE capture_session_done_during_task;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE table capture_session_done_during_task.t (id int primary key auto_increment, a int)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "CREATE DATABASE capture_session_done_during_task;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - run_sql "CREATE table capture_session_done_during_task.t (id int primary key auto_increment, a int)" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "CREATE DATABASE capture_session_done_during_task;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + run_sql "CREATE table capture_session_done_during_task.t (id int primary key auto_increment, a int)" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_sql "INSERT INTO capture_session_done_during_task.t values (),(),(),(),(),(),()" ${UP_TIDB_HOST} ${UP_TIDB_PORT} export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/processorManagerHandleNewChangefeedDelay=sleep(2000)' @@ -53,7 +53,7 @@ function run() { # capture handle task delays 10s, minus 2s wait task dispatched sleep 1 - check_table_exists "capture_session_done_during_task.t" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "capture_session_done_during_task.t" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql "INSERT INTO capture_session_done_during_task.t values (),(),(),(),(),(),()" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/capture_suicide_while_balance_table/run.sh b/tests/integration_tests/capture_suicide_while_balance_table/run.sh index 402ae21dfb1..bcc97700c8b 100644 --- a/tests/integration_tests/capture_suicide_while_balance_table/run.sh +++ b/tests/integration_tests/capture_suicide_while_balance_table/run.sh @@ -48,7 +48,7 @@ function run() { done for i in $(seq 1 4); do - check_table_exists "capture_suicide_while_balance_table.t$i" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "capture_suicide_while_balance_table.t$i" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done capture1_id=$(cdc cli capture list | jq -r '.[]|select(.address=="127.0.0.1:8300")|.id') diff --git a/tests/integration_tests/changefeed_auto_stop/run.sh b/tests/integration_tests/changefeed_auto_stop/run.sh index 430c4882ef0..5efb84c98aa 100755 --- a/tests/integration_tests/changefeed_auto_stop/run.sh +++ b/tests/integration_tests/changefeed_auto_stop/run.sh @@ -48,7 +48,7 @@ function run() { ensure $MAX_RETRIES check_changefeed_state "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" ${changefeedid} "normal" "null" "" for i in $(seq $DB_COUNT); do - check_table_exists "changefeed_auto_stop_$i.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "changefeed_auto_stop_$i.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/changefeed_dup_error_restart/run.sh b/tests/integration_tests/changefeed_dup_error_restart/run.sh index e0756731a91..7ff430c22b9 100755 --- a/tests/integration_tests/changefeed_dup_error_restart/run.sh +++ b/tests/integration_tests/changefeed_dup_error_restart/run.sh @@ -36,13 +36,13 @@ function run() { run_sql "CREATE TABLE changefeed_dup_error_restart.finish_mark_1 (a int primary key);" sleep 30 - check_table_exists "changefeed_dup_error_restart.finish_mark_1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + check_table_exists "changefeed_dup_error_restart.finish_mark_1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 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=changefeed_dup_error_restart run_sql "CREATE TABLE changefeed_dup_error_restart.finish_mark_2 (a int primary key);" sleep 30 - check_table_exists "changefeed_dup_error_restart.finish_mark_2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + check_table_exists "changefeed_dup_error_restart.finish_mark_2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/changefeed_error/run.sh b/tests/integration_tests/changefeed_error/run.sh index d4ada6a2f91..7d386a595d5 100755 --- a/tests/integration_tests/changefeed_error/run.sh +++ b/tests/integration_tests/changefeed_error/run.sh @@ -63,7 +63,7 @@ function run() { ensure $MAX_RETRIES check_changefeed_state http://${UP_PD_HOST_1}:${UP_PD_PORT_1} ${changefeedid} "failed" "[CDC:ErrStartTsBeforeGC]" "" run_cdc_cli changefeed resume -c $changefeedid - check_table_exists "changefeed_error.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "changefeed_error.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 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=changefeed_error diff --git a/tests/integration_tests/changefeed_pause_resume/run.sh b/tests/integration_tests/changefeed_pause_resume/run.sh index cbd3fbc2459..804648ec1a2 100755 --- a/tests/integration_tests/changefeed_pause_resume/run.sh +++ b/tests/integration_tests/changefeed_pause_resume/run.sh @@ -42,7 +42,7 @@ function run() { for i in $(seq 1 $TABLE_COUNT); do table="changefeed_pause_resume.t$i" - check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done for i in $(seq 1 10); do diff --git a/tests/integration_tests/changefeed_reconstruct/run.sh b/tests/integration_tests/changefeed_reconstruct/run.sh index b9790b53920..4ce2a708266 100755 --- a/tests/integration_tests/changefeed_reconstruct/run.sh +++ b/tests/integration_tests/changefeed_reconstruct/run.sh @@ -60,7 +60,7 @@ function run() { run_sql "CREATE DATABASE changefeed_reconstruct;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} 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=changefeed_reconstruct - check_table_exists "changefeed_reconstruct.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "changefeed_reconstruct.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml # kill capture @@ -81,7 +81,7 @@ function run() { run_sql "CREATE DATABASE changefeed_reconstruct;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} 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=changefeed_reconstruct - check_table_exists "changefeed_reconstruct.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "changefeed_reconstruct.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/changefeed_resume_with_checkpoint_ts/run.sh b/tests/integration_tests/changefeed_resume_with_checkpoint_ts/run.sh index 378e218142b..04f1d8da42d 100644 --- a/tests/integration_tests/changefeed_resume_with_checkpoint_ts/run.sh +++ b/tests/integration_tests/changefeed_resume_with_checkpoint_ts/run.sh @@ -31,7 +31,7 @@ function resume_changefeed_in_stopped_state() { done for i in $(seq 1 2); do table="test.table$i" - check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done check_sync_diff $WORK_DIR $CUR/conf/diff_config2.toml cdc cli changefeed pause --changefeed-id=$changefeed_id --pd=$pd_addr @@ -55,11 +55,11 @@ function resume_changefeed_in_stopped_state() { cdc cli changefeed resume --changefeed-id=$changefeed_id --pd=$pd_addr --overwrite-checkpoint-ts=$checkpointTs3 --no-confirm=true for i in $(seq 5 6); do table="test.table$i" - check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done stmt="select count(*) as table_count from information_schema.tables where table_schema='test'" - run_sql "$stmt" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && check_contains "table_count: 4" + run_sql "$stmt" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "table_count: 4" run_sql "$stmt" ${UP_TIDB_HOST} ${UP_TIDB_PORT} && check_contains "table_count: 6" check_sync_diff $WORK_DIR $CUR/conf/diff_config1.toml @@ -68,9 +68,9 @@ function resume_changefeed_in_stopped_state() { cdc cli changefeed resume --changefeed-id=$changefeed_id --pd=$pd_addr --overwrite-checkpoint-ts=$checkpointTs1 --no-confirm=true for i in $(seq 3 4); do table="test.table$i" - check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done - run_sql "$stmt" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && check_contains "table_count: 6" + run_sql "$stmt" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "table_count: 6" run_sql "$stmt" ${UP_TIDB_HOST} ${UP_TIDB_PORT} && check_contains "table_count: 6" check_sync_diff $WORK_DIR $CUR/conf/diff_config2.toml @@ -96,7 +96,7 @@ function resume_changefeed_in_failed_state() { done for i in $(seq 1 2); do table="test1.table$i" - check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done check_sync_diff $WORK_DIR $CUR/conf/diff_config3.toml diff --git a/tests/integration_tests/charset_gbk/run.sh b/tests/integration_tests/charset_gbk/run.sh index b16316bbf11..20d8a5795c9 100755 --- a/tests/integration_tests/charset_gbk/run.sh +++ b/tests/integration_tests/charset_gbk/run.sh @@ -40,10 +40,10 @@ function run() { run_sql_file $CUR/data/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists charset_gbk_test0.t0 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists charset_gbk_test0.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists charset_gbk_test1.t0 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists charset_gbk_test0.t0 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists charset_gbk_test0.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists charset_gbk_test1.t0 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} echo "check table exists success" check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 60 diff --git a/tests/integration_tests/ci_collation_compatibility/run.sh b/tests/integration_tests/ci_collation_compatibility/run.sh index ddb4d6d3d93..06ddcb28b6c 100755 --- a/tests/integration_tests/ci_collation_compatibility/run.sh +++ b/tests/integration_tests/ci_collation_compatibility/run.sh @@ -48,8 +48,8 @@ function run() { run_sql "CREATE TABLE ci_collation_compatibility.finish_mark_1 (a int primary key);" sleep 30 - check_table_exists "ci_collation_compatibility.t" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "ci_collation_compatibility.finish_mark_1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + check_table_exists "ci_collation_compatibility.t" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "ci_collation_compatibility.finish_mark_1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/cli_tls_with_auth/run.sh b/tests/integration_tests/cli_tls_with_auth/run.sh index b71c0efc12a..3d8349ad87d 100644 --- a/tests/integration_tests/cli_tls_with_auth/run.sh +++ b/tests/integration_tests/cli_tls_with_auth/run.sh @@ -95,8 +95,8 @@ function run() { esac # Make sure changefeed is created. - check_table_exists test.simple ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists test."\`simple-dash\`" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists test.simple ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists test."\`simple-dash\`" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_changefeed_state "https://${TLS_PD_HOST}:${TLS_PD_PORT}" $uuid "normal" "null" "" $TLS_DIR diff --git a/tests/integration_tests/cli_with_auth/run.sh b/tests/integration_tests/cli_with_auth/run.sh index 5d496f57581..e3a714301cb 100644 --- a/tests/integration_tests/cli_with_auth/run.sh +++ b/tests/integration_tests/cli_with_auth/run.sh @@ -65,8 +65,8 @@ function run() { esac # Make sure changefeed is created. - check_table_exists test.simple ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists test."\`simple-dash\`" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists test.simple ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists test."\`simple-dash\`" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_changefeed_state "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" $uuid "normal" "null" "" diff --git a/tests/integration_tests/clustered_index/run.sh b/tests/integration_tests/clustered_index/run.sh index a42a0b79c4c..b7acfe0d81b 100755 --- a/tests/integration_tests/clustered_index/run.sh +++ b/tests/integration_tests/clustered_index/run.sh @@ -42,9 +42,9 @@ function run() { run_sql_file $CUR/data/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists clustered_index_test.t0 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists clustered_index_test.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists clustered_index_test.t2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists clustered_index_test.t0 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists clustered_index_test.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists clustered_index_test.t2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} echo "check table exists success" check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 60 diff --git a/tests/integration_tests/common_1/run.sh b/tests/integration_tests/common_1/run.sh index 72ac63f7cd5..afb1d34ecd0 100644 --- a/tests/integration_tests/common_1/run.sh +++ b/tests/integration_tests/common_1/run.sh @@ -33,7 +33,7 @@ function run() { # feature by default, turn it on first run_sql "set global tidb_enable_change_multi_schema = on" ${UP_TIDB_HOST} ${UP_TIDB_PORT} # This must be set before cdc server starts - run_sql "set global tidb_enable_change_multi_schema = on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set global tidb_enable_change_multi_schema = on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} # TiDB global variables cache 2 seconds at most sleep 2 fi @@ -84,9 +84,9 @@ EOF run_sql_file $CUR/data/test_finish.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists common_1.v1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists common_1.recover_and_insert ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists common_1.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists common_1.v1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists common_1.recover_and_insert ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists common_1.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/consistent_partition_table/run.sh b/tests/integration_tests/consistent_partition_table/run.sh index b5a21eeb651..ac96d9deca9 100644 --- a/tests/integration_tests/consistent_partition_table/run.sh +++ b/tests/integration_tests/consistent_partition_table/run.sh @@ -13,7 +13,7 @@ mkdir -p "$WORK_DIR" stop() { # to distinguish whether the test failed in the DML synchronization phase or the DDL synchronization phase - echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -e "show databases;") + echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT_1} -uroot -e "show databases;") stop_tidb_cluster } @@ -26,7 +26,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix partition_table.server1 @@ -34,7 +34,7 @@ function run() { changefeed_id=$(cdc cli changefeed create --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') run_sql_file $CUR/data/create.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "partition_table2.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + check_table_exists "partition_table2.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 sleep 5 cleanup_process $CDC_BINARY # Inject the failpoint to prevent sink execution, but the global resolved can be moved forward. @@ -59,7 +59,7 @@ function run() { --storage="$storage_path" \ --sink-uri="mysql://normal:123456@127.0.0.1:3306/" - # check_table_exists "partition_table.finish_mark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + # check_table_exists "partition_table.finish_mark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 check_sync_diff $WORK_DIR $WORK_DIR/diff_config.toml } diff --git a/tests/integration_tests/consistent_replicate_ddl/run.sh b/tests/integration_tests/consistent_replicate_ddl/run.sh index bdb1c1c99a5..b2537ae81f3 100644 --- a/tests/integration_tests/consistent_replicate_ddl/run.sh +++ b/tests/integration_tests/consistent_replicate_ddl/run.sh @@ -13,7 +13,7 @@ mkdir -p "$WORK_DIR" stop() { # to distinguish whether the test failed in the DML synchronization phase or the DDL synchronization phase - echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -e "SELECT count(*) FROM consistent_replicate_ddl.usertable;") + echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT_1} -uroot -e "SELECT count(*) FROM consistent_replicate_ddl.usertable;") stop_tidb_cluster } @@ -26,7 +26,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix consistent_replicate_ddl.server1 @@ -39,10 +39,10 @@ function run() { run_sql "CREATE TABLE consistent_replicate_ddl.usertable2 like consistent_replicate_ddl.usertable" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE TABLE consistent_replicate_ddl.usertable3 like consistent_replicate_ddl.usertable" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE TABLE consistent_replicate_ddl.usertable_bak like consistent_replicate_ddl.usertable" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "consistent_replicate_ddl.usertable1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "consistent_replicate_ddl.usertable2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 - check_table_exists "consistent_replicate_ddl.usertable3" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 - check_table_exists "consistent_replicate_ddl.usertable_bak" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + check_table_exists "consistent_replicate_ddl.usertable1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "consistent_replicate_ddl.usertable2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 + check_table_exists "consistent_replicate_ddl.usertable3" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 + check_table_exists "consistent_replicate_ddl.usertable_bak" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 sleep 5 cleanup_process $CDC_BINARY # Inject the failpoint to prevent sink execution, but the global resolved can be moved forward. @@ -96,7 +96,7 @@ function run() { --storage="$storage_path" \ --sink-uri="mysql://normal:123456@127.0.0.1:3306/" - check_table_exists "consistent_replicate_ddl.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + check_table_exists "consistent_replicate_ddl.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 check_sync_diff $WORK_DIR $WORK_DIR/diff_config.toml } diff --git a/tests/integration_tests/consistent_replicate_gbk/run.sh b/tests/integration_tests/consistent_replicate_gbk/run.sh index 1359e3abc7c..93a6073055a 100644 --- a/tests/integration_tests/consistent_replicate_gbk/run.sh +++ b/tests/integration_tests/consistent_replicate_gbk/run.sh @@ -60,8 +60,8 @@ function run() { run_sql "CREATE TABLE consistent_replicate_gbk.GBKTABLE (id INT,name varchar(128),country char(32),city varchar(64),description text,image tinyblob,PRIMARY KEY (id)) ENGINE = InnoDB CHARSET = gbk;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO consistent_replicate_gbk.GBKTABLE VALUES (1, '测试', '中国', '上海', '你好,世界', 0xC4E3BAC3CAC0BDE7);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE table consistent_replicate_gbk.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "consistent_replicate_gbk.GBKTABLE" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "consistent_replicate_gbk.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + check_table_exists "consistent_replicate_gbk.GBKTABLE" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "consistent_replicate_gbk.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml # Inject the failpoint to prevent sink execution, but the global resolved can be moved forward. diff --git a/tests/integration_tests/consistent_replicate_nfs/run.sh b/tests/integration_tests/consistent_replicate_nfs/run.sh index f2b9a6f6612..9083b72a296 100644 --- a/tests/integration_tests/consistent_replicate_nfs/run.sh +++ b/tests/integration_tests/consistent_replicate_nfs/run.sh @@ -10,7 +10,7 @@ SINK_TYPE=$1 stop() { # to distinguish whether the test failed in the DML synchronization phase or the DDL synchronization phase - echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -e "select count(*) from consistent_replicate_nfs.usertable;") + echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT_1} -uroot -e "select count(*) from consistent_replicate_nfs.usertable;") stop_tidb_cluster } @@ -33,7 +33,7 @@ function run() { run_sql "CREATE DATABASE consistent_replicate_nfs;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} 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=consistent_replicate_nfs - check_table_exists "consistent_replicate_nfs.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "consistent_replicate_nfs.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} cleanup_process $CDC_BINARY # to ensure row changed events have been replicated to TiCDC diff --git a/tests/integration_tests/consistent_replicate_storage_file/run.sh b/tests/integration_tests/consistent_replicate_storage_file/run.sh index 5e03d111e43..e599a34f31d 100644 --- a/tests/integration_tests/consistent_replicate_storage_file/run.sh +++ b/tests/integration_tests/consistent_replicate_storage_file/run.sh @@ -13,7 +13,7 @@ mkdir -p "$WORK_DIR" stop() { # to distinguish whether the test failed in the DML synchronization phase or the DDL synchronization phase - echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -e "select count(*) from consistent_replicate_storage_file.usertable;") + echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT_1} -uroot -e "select count(*) from consistent_replicate_storage_file.usertable;") stop_tidb_cluster } @@ -26,7 +26,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY @@ -37,9 +37,9 @@ function run() { 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=consistent_replicate_storage_file run_sql "CREATE table consistent_replicate_storage_file.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "consistent_replicate_storage_file.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "consistent_replicate_storage_file.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 - check_table_exists "consistent_replicate_storage_file.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + check_table_exists "consistent_replicate_storage_file.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "consistent_replicate_storage_file.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 + check_table_exists "consistent_replicate_storage_file.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/consistent_replicate_storage_file_large_value/run.sh b/tests/integration_tests/consistent_replicate_storage_file_large_value/run.sh index 591db8bfb38..406d6da77e4 100644 --- a/tests/integration_tests/consistent_replicate_storage_file_large_value/run.sh +++ b/tests/integration_tests/consistent_replicate_storage_file_large_value/run.sh @@ -13,7 +13,7 @@ mkdir -p "$WORK_DIR" stop() { # to distinguish whether the test failed in the DML synchronization phase or the DDL synchronization phase - echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -e "select count(*) from consistent_replicate_storage_file_large_value.usertable;") + echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT_1} -uroot -e "select count(*) from consistent_replicate_storage_file_large_value.usertable;") stop_tidb_cluster } @@ -26,7 +26,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY @@ -37,9 +37,9 @@ function run() { 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=consistent_replicate_storage_file_large_value run_sql "CREATE table consistent_replicate_storage_file_large_value.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "consistent_replicate_storage_file_large_value.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "consistent_replicate_storage_file_large_value.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 - check_table_exists "consistent_replicate_storage_file_large_value.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + check_table_exists "consistent_replicate_storage_file_large_value.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "consistent_replicate_storage_file_large_value.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 + check_table_exists "consistent_replicate_storage_file_large_value.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/consistent_replicate_storage_s3/run.sh b/tests/integration_tests/consistent_replicate_storage_s3/run.sh index 4ec5d429f26..0e464b5db65 100644 --- a/tests/integration_tests/consistent_replicate_storage_s3/run.sh +++ b/tests/integration_tests/consistent_replicate_storage_s3/run.sh @@ -36,7 +36,7 @@ stop_minio() { stop() { # to distinguish whether the test failed in the DML synchronization phase or the DDL synchronization phase - echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -e "select count(*) from consistent_replicate_storage_s3.usertable;") + echo $(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT_1} -uroot -e "select count(*) from consistent_replicate_storage_s3.usertable;") stop_minio stop_tidb_cluster } @@ -52,7 +52,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY @@ -63,9 +63,9 @@ function run() { 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=consistent_replicate_storage_s3 run_sql "CREATE table consistent_replicate_storage_s3.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "consistent_replicate_storage_s3.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "consistent_replicate_storage_s3.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 - check_table_exists "consistent_replicate_storage_s3.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 120 + check_table_exists "consistent_replicate_storage_s3.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "consistent_replicate_storage_s3.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 + check_table_exists "consistent_replicate_storage_s3.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 120 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/csv_storage_multi_tables_ddl/run.sh b/tests/integration_tests/csv_storage_multi_tables_ddl/run.sh index ec75a5d761f..a3808ef7fc1 100755 --- a/tests/integration_tests/csv_storage_multi_tables_ddl/run.sh +++ b/tests/integration_tests/csv_storage_multi_tables_ddl/run.sh @@ -45,12 +45,12 @@ function run() { run_storage_consumer $WORK_DIR $SINK_URI3 "$CUR/conf/error-2.toml" 3 run_sql_file $CUR/data/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t55 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t66 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t7 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t88 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists multi_tables_ddl_test.t55 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists multi_tables_ddl_test.t66 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists multi_tables_ddl_test.t7 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists multi_tables_ddl_test.t88 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists multi_tables_ddl_test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists multi_tables_ddl_test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} echo "check table exists success" # changefeed test-error will not report an error, "multi_tables_ddl_test.t555 to multi_tables_ddl_test.t55" patr will be skipped. diff --git a/tests/integration_tests/csv_storage_partition_table/run.sh b/tests/integration_tests/csv_storage_partition_table/run.sh index c5acb21a83a..e1cbef8d476 100644 --- a/tests/integration_tests/csv_storage_partition_table/run.sh +++ b/tests/integration_tests/csv_storage_partition_table/run.sh @@ -17,7 +17,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --logsuffix 0 run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8301" --logsuffix 1 @@ -34,10 +34,10 @@ function run() { sleep 8 # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists partition_table.t ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists partition_table.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - # check_table_exists partition_table.t2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists partition_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists partition_table.t ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists partition_table.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + # check_table_exists partition_table.t2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists partition_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/csv_storage_update_pk_clustered/run.sh b/tests/integration_tests/csv_storage_update_pk_clustered/run.sh index 8606518da0a..8a8c03dcf40 100644 --- a/tests/integration_tests/csv_storage_update_pk_clustered/run.sh +++ b/tests/integration_tests/csv_storage_update_pk_clustered/run.sh @@ -24,7 +24,7 @@ function run_changefeed() { echo "expected split count $expected_split_count, real split count $real_split_count" exit 1 fi - run_sql "drop database if exists test" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "drop database if exists test" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} } function run() { diff --git a/tests/integration_tests/csv_storage_update_pk_nonclustered/run.sh b/tests/integration_tests/csv_storage_update_pk_nonclustered/run.sh index adbf3392ffd..022bd3e6046 100644 --- a/tests/integration_tests/csv_storage_update_pk_nonclustered/run.sh +++ b/tests/integration_tests/csv_storage_update_pk_nonclustered/run.sh @@ -32,7 +32,7 @@ function run_changefeed() { echo "expected split count $expected_split_count, real split count $real_split_count" exit 1 fi - run_sql "drop database if exists test" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "drop database if exists test" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} } function run() { diff --git a/tests/integration_tests/ddl_attributes/run.sh b/tests/integration_tests/ddl_attributes/run.sh index d9570e43a90..6585f9510c6 100644 --- a/tests/integration_tests/ddl_attributes/run.sh +++ b/tests/integration_tests/ddl_attributes/run.sh @@ -41,8 +41,8 @@ function run() { run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first sleep 3 - check_table_exists ddl_attributes.attributes_t1_new ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists ddl_attributes.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists ddl_attributes.attributes_t1_new ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists ddl_attributes.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY @@ -50,25 +50,25 @@ function run() { run_sql "show create table ddl_attributes.placement_t1;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "show create table ddl_attributes.placement_t2;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "show create table ddl_attributes.placement_t1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "show create table ddl_attributes.placement_t1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "CREATE TABLE \`placement_t1\` " - run_sql "show create table ddl_attributes.placement_t2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "show create table ddl_attributes.placement_t2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "CREATE TABLE \`placement_t2\` " TTL_MARK='![ttl]' CREATE_TTL_SQL_CONTAINS1="/*T${TTL_MARK} TTL=\`t\` + INTERVAL 1 DAY */ /*T${TTL_MARK} TTL_ENABLE='OFF' */ /*T${TTL_MARK} TTL_JOB_INTERVAL='1h' */" CREATE_TTL_SQL_CONTAINS2="/*T${TTL_MARK} TTL=\`t\` + INTERVAL 1 DAY */ /*T${TTL_MARK} TTL_ENABLE='OFF' */ /*T${TTL_MARK} TTL_JOB_INTERVAL='7h' */" - run_sql "show create table ddl_attributes.ttl_t1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "show create table ddl_attributes.ttl_t1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "$CREATE_TTL_SQL_CONTAINS1" - run_sql "show create table ddl_attributes.ttl_t2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "show create table ddl_attributes.ttl_t2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "$CREATE_TTL_SQL_CONTAINS1" - run_sql "show create table ddl_attributes.ttl_t3;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "show create table ddl_attributes.ttl_t3;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "$CREATE_TTL_SQL_CONTAINS1" - run_sql "show create table ddl_attributes.ttl_t4;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "show create table ddl_attributes.ttl_t4;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_not_contains "TTL_ENABLE" && check_not_contains "TTL=" - run_sql "show create table ddl_attributes.ttl_t5;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "show create table ddl_attributes.ttl_t5;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "$CREATE_TTL_SQL_CONTAINS2" } diff --git a/tests/integration_tests/ddl_manager/run.sh b/tests/integration_tests/ddl_manager/run.sh index 609db1893c6..e72529cc302 100755 --- a/tests/integration_tests/ddl_manager/run.sh +++ b/tests/integration_tests/ddl_manager/run.sh @@ -48,7 +48,7 @@ function run() { owner_pid=$(ps -C $CDC_BINARY -o pid= | awk '{print $1}') run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - check_table_exists ddl_manager.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists ddl_manager.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 # make sure all tables are equal in upstream and downstream check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 180 cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/ddl_only_block_related_table/run.sh b/tests/integration_tests/ddl_only_block_related_table/run.sh index f104b540e47..27f7ae8b575 100755 --- a/tests/integration_tests/ddl_only_block_related_table/run.sh +++ b/tests/integration_tests/ddl_only_block_related_table/run.sh @@ -80,7 +80,7 @@ function run() { run_sql_file $CUR/data/start.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists ddl_only_block_related_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists ddl_only_block_related_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} kill_cdc_pid $owner_pid export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/ExecuteNotDone=return(true)' diff --git a/tests/integration_tests/ddl_puller_lag/run.sh b/tests/integration_tests/ddl_puller_lag/run.sh index 5275936bf44..de720ec2922 100644 --- a/tests/integration_tests/ddl_puller_lag/run.sh +++ b/tests/integration_tests/ddl_puller_lag/run.sh @@ -46,7 +46,7 @@ function sql_check() { # the following statement will be not executed # check table ddl_puller_lag1. - run_sql "SELECT id, val FROM test.ddl_puller_lag1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "SELECT id, val FROM test.ddl_puller_lag1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "id: 1" && check_contains "val: 1" && check_contains "id: 2" && @@ -54,7 +54,7 @@ function sql_check() { check_not_contains "id: 3" && # check table ddl_puller_lag2. - run_sql "SELECT id, val FROM test.ddl_puller_lag2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "SELECT id, val FROM test.ddl_puller_lag2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "id: 1" && check_contains "val: 1" && check_contains "id: 2" && diff --git a/tests/integration_tests/ddl_reentrant/run.sh b/tests/integration_tests/ddl_reentrant/run.sh index 50dc3198ae1..3d668ad3dc5 100644 --- a/tests/integration_tests/ddl_reentrant/run.sh +++ b/tests/integration_tests/ddl_reentrant/run.sh @@ -118,8 +118,8 @@ function run() { run_sql "set @@global.tidb_enable_exchange_partition=on" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "set global tidb_enable_change_multi_schema = on" ${UP_TIDB_HOST} ${UP_TIDB_PORT} # This must be set before cdc server starts - run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - run_sql "set global tidb_enable_change_multi_schema = on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + run_sql "set global tidb_enable_change_multi_schema = on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} # TiDB global variables cache 2 seconds at most sleep 2 fi diff --git a/tests/integration_tests/ddl_sequence/run.sh b/tests/integration_tests/ddl_sequence/run.sh index d9730e46ec8..e82ce9bf89e 100644 --- a/tests/integration_tests/ddl_sequence/run.sh +++ b/tests/integration_tests/ddl_sequence/run.sh @@ -38,7 +38,7 @@ function run() { esac run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists ddl_sequence.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists ddl_sequence.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/default_value/run.sh b/tests/integration_tests/default_value/run.sh index e8e7d0de5aa..6f79f958ee8 100755 --- a/tests/integration_tests/default_value/run.sh +++ b/tests/integration_tests/default_value/run.sh @@ -52,11 +52,11 @@ if [ "$SINK_TYPE" != "storage" ]; then cd "$(dirname "$0")" set -o pipefail GO111MODULE=on go run main.go -config ./config.toml 2>&1 | tee $WORK_DIR/tester.log - check_table_exists mark.finish_mark_1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists mark.finish_mark_1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 # ticdc cost too much sink DDL, just leave more time here - check_table_exists mark.finish_mark_3 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 900 - check_table_exists mark.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists mark.finish_mark_3 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 900 + check_table_exists mark.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 check_sync_diff $WORK_DIR $CUR/diff_config.toml cleanup_process $CDC_BINARY check_logs $WORK_DIR diff --git a/tests/integration_tests/drop_many_tables/run.sh b/tests/integration_tests/drop_many_tables/run.sh index 74d6659442c..ef83850fdf1 100644 --- a/tests/integration_tests/drop_many_tables/run.sh +++ b/tests/integration_tests/drop_many_tables/run.sh @@ -38,7 +38,7 @@ function run() { esac run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists drop_tables.c ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists drop_tables.c ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/event_filter/run.sh b/tests/integration_tests/event_filter/run.sh index 97ee677ed21..245ca0841df 100644 --- a/tests/integration_tests/event_filter/run.sh +++ b/tests/integration_tests/event_filter/run.sh @@ -40,29 +40,29 @@ function run() { # make suer table t1 is deleted in upstream and exists in downstream check_table_not_exists "event_filter.t1" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "event_filter.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "event_filter.t_normal" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "event_filter.t_truncate" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "event_filter.t_alter" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "event_filter.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "event_filter.t_normal" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "event_filter.t_truncate" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "event_filter.t_alter" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} # check those rows that are not filtered are synced to downstream - run_sql "select count(1) from event_filter.t1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "select count(1) from event_filter.t1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_contains "count(1): 2" - run_sql "select count(2) from event_filter.t1 where id=1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "select count(2) from event_filter.t1 where id=1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_contains "count(2): 1" - run_sql "select count(3) from event_filter.t1 where id=2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "select count(3) from event_filter.t1 where id=2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_contains "count(3): 0" - run_sql "select count(4) from event_filter.t1 where id=3;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "select count(4) from event_filter.t1 where id=3;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_contains "count(4): 0" - run_sql "select count(5) from event_filter.t1 where id=4;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "select count(5) from event_filter.t1 where id=4;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_contains "count(5): 1" - run_sql "TRUNCATE TABLE event_filter.t_truncate;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "TRUNCATE TABLE event_filter.t_truncate;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_sql_file $CUR/data/test_truncate.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "ALTER TABLE event_filter.t_alter MODIFY t_bigint BIGINT;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "ALTER TABLE event_filter.t_alter MODIFY t_bigint BIGINT;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_sql_file $CUR/data/test_alter.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "create table event_filter.finish_mark(id int primary key);" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "event_filter.finish_mark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "create table event_filter.finish_mark(id int primary key);" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "event_filter.finish_mark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} # check table t_normal is replicated check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/force_replicate_table/run.sh b/tests/integration_tests/force_replicate_table/run.sh index 2636f3130f4..798f3dbf2ce 100755 --- a/tests/integration_tests/force_replicate_table/run.sh +++ b/tests/integration_tests/force_replicate_table/run.sh @@ -79,13 +79,13 @@ function run() { run_sql_file $CUR/data/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} for i in $(seq 0 6); do table="force_replicate_table.t$i" - check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done # data could be duplicated due to https://github.com/pingcap/tiflow/issues/964, # so we just check downstream contains all data in upstream. for i in $(seq 0 6); do ensure 10 check_data_subset "force_replicate_table.t$i" \ - ${UP_TIDB_HOST} ${UP_TIDB_PORT} ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + ${UP_TIDB_HOST} ${UP_TIDB_PORT} ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done cleanup_process $CDC_BINARY } diff --git a/tests/integration_tests/foreign_key/run.sh b/tests/integration_tests/foreign_key/run.sh index 43c522437a8..3bfb1927d8b 100644 --- a/tests/integration_tests/foreign_key/run.sh +++ b/tests/integration_tests/foreign_key/run.sh @@ -38,7 +38,7 @@ function run() { esac run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists foreign_key.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists foreign_key.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/generate_column/run.sh b/tests/integration_tests/generate_column/run.sh index 5b251d9efd1..b8470670dd2 100644 --- a/tests/integration_tests/generate_column/run.sh +++ b/tests/integration_tests/generate_column/run.sh @@ -48,8 +48,8 @@ function run() { esac run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists generate_column.t ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists generate_column.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists generate_column.t ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists generate_column.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/http_api/run.sh b/tests/integration_tests/http_api/run.sh index db56040660d..b7a876a70b5 100644 --- a/tests/integration_tests/http_api/run.sh +++ b/tests/integration_tests/http_api/run.sh @@ -51,7 +51,7 @@ function run() { # wait for above sql done in the up source sleep 2 - check_table_exists test.simple1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists test.simple1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sequential_cases1=( "list_changefeed" diff --git a/tests/integration_tests/http_api_tls/run.sh b/tests/integration_tests/http_api_tls/run.sh index a88fcfff6fb..4f5e881ad43 100644 --- a/tests/integration_tests/http_api_tls/run.sh +++ b/tests/integration_tests/http_api_tls/run.sh @@ -103,7 +103,7 @@ function run() { # wait for above sql done in the up source sleep 2 - check_table_exists test.verify_table_eligible ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists test.verify_table_eligible ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sequential_cases=( "list_changefeed" diff --git a/tests/integration_tests/http_api_tls_with_user_auth/run.sh b/tests/integration_tests/http_api_tls_with_user_auth/run.sh index 327ba9b790f..281d8066af0 100644 --- a/tests/integration_tests/http_api_tls_with_user_auth/run.sh +++ b/tests/integration_tests/http_api_tls_with_user_auth/run.sh @@ -109,7 +109,7 @@ function run() { # wait for above sql done in the up source sleep 2 - check_table_exists test.simple1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists test.simple1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sequential_cases=( "list_changefeed" diff --git a/tests/integration_tests/kafka_big_messages/run.sh b/tests/integration_tests/kafka_big_messages/run.sh index d0904360006..02acd750802 100755 --- a/tests/integration_tests/kafka_big_messages/run.sh +++ b/tests/integration_tests/kafka_big_messages/run.sh @@ -43,7 +43,7 @@ function run() { run_sql_file $CUR/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} table="kafka_big_messages.test" - check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/kafka_big_messages_v2/run.sh b/tests/integration_tests/kafka_big_messages_v2/run.sh index d1f861fdeee..afc6882b8f7 100755 --- a/tests/integration_tests/kafka_big_messages_v2/run.sh +++ b/tests/integration_tests/kafka_big_messages_v2/run.sh @@ -43,7 +43,7 @@ function run() { run_sql_file $CUR/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} table="kafka_big_messages.test" - check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/kafka_column_selector/run.sh b/tests/integration_tests/kafka_column_selector/run.sh index e8bd3a83f89..69dc6fe99df 100644 --- a/tests/integration_tests/kafka_column_selector/run.sh +++ b/tests/integration_tests/kafka_column_selector/run.sh @@ -39,9 +39,9 @@ function run() { GO111MODULE=on go build fi - check_table_exists "test1.finishmark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test1.finishmark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} - ./checksum_checker --upstream-uri "root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/" --downstream-uri "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT})/" --databases "test,test1" --config="$CUR/conf/changefeed.toml" + ./checksum_checker --upstream-uri "root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/" --downstream-uri "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1})/" --databases "test,test1" --config="$CUR/conf/changefeed.toml" cleanup_process $CDC_BINARY } diff --git a/tests/integration_tests/kafka_column_selector_avro/run.sh b/tests/integration_tests/kafka_column_selector_avro/run.sh index e98fc71fc17..02840029f0e 100644 --- a/tests/integration_tests/kafka_column_selector_avro/run.sh +++ b/tests/integration_tests/kafka_column_selector_avro/run.sh @@ -53,9 +53,9 @@ function run() { GO111MODULE=on go build fi - check_table_exists "test.finishmark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test.finishmark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} - ./checksum_checker --upstream-uri "root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/" --downstream-uri "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT})/" --databases "test" --config="$CUR/conf/changefeed.toml" + ./checksum_checker --upstream-uri "root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/" --downstream-uri "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1})/" --databases "test" --config="$CUR/conf/changefeed.toml" cleanup_process $CDC_BINARY } diff --git a/tests/integration_tests/kafka_compression/run.sh b/tests/integration_tests/kafka_compression/run.sh index 64f5544781e..f446fd9962b 100644 --- a/tests/integration_tests/kafka_compression/run.sh +++ b/tests/integration_tests/kafka_compression/run.sh @@ -23,7 +23,7 @@ function test_compression() { echo "can't found producer compression algorithm" exit 1 fi - check_table_exists test.$1_finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.$1_finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_cdc_cli changefeed pause -c $1 run_cdc_cli changefeed remove -c $1 diff --git a/tests/integration_tests/kafka_messages/run.sh b/tests/integration_tests/kafka_messages/run.sh index 88562a92725..d8eb836c011 100755 --- a/tests/integration_tests/kafka_messages/run.sh +++ b/tests/integration_tests/kafka_messages/run.sh @@ -38,26 +38,26 @@ function run_length_limit() { # Add a check table to reduce check time, or if we check data with sync diff # directly, there maybe a lot of diff data at first because of the incremental scan run_sql "CREATE table kafka_message.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "kafka_message.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 - check_table_exists "kafka_message.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 + check_table_exists "kafka_message.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 + check_table_exists "kafka_message.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql "truncate table kafka_message.usertable" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql "CREATE table kafka_message.check2(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "kafka_message.check2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 + check_table_exists "kafka_message.check2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 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=kafka_message run_sql "CREATE table kafka_message.check3(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "kafka_message.check3" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 + check_table_exists "kafka_message.check3" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql "create table kafka_message.usertable2 like kafka_message.usertable" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "insert into kafka_message.usertable2 select * from kafka_message.usertable" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "create table kafka_message.check4(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "kafka_message.usertable2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 - check_table_exists "kafka_message.check4" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 + check_table_exists "kafka_message.usertable2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 + check_table_exists "kafka_message.check4" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml @@ -89,26 +89,26 @@ function run_batch_size_limit() { # Add a check table to reduce check time, or if we check data with sync diff # directly, there maybe a lot of diff data at first because of the incremental scan run_sql "CREATE table kafka_message.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "kafka_message.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 - check_table_exists "kafka_message.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 + check_table_exists "kafka_message.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 + check_table_exists "kafka_message.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql "truncate table kafka_message.usertable" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql "CREATE table kafka_message.check2(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "kafka_message.check2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 + check_table_exists "kafka_message.check2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 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=kafka_message run_sql "CREATE table kafka_message.check3(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "kafka_message.check3" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 + check_table_exists "kafka_message.check3" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql "create table kafka_message.usertable2 like kafka_message.usertable" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "insert into kafka_message.usertable2 select * from kafka_message.usertable" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "create table kafka_message.check4(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "kafka_message.usertable2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 - check_table_exists "kafka_message.check4" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 + check_table_exists "kafka_message.usertable2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 + check_table_exists "kafka_message.check4" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/kafka_simple_basic/run.sh b/tests/integration_tests/kafka_simple_basic/run.sh index 23e1b40a04f..6d468ed531e 100644 --- a/tests/integration_tests/kafka_simple_basic/run.sh +++ b/tests/integration_tests/kafka_simple_basic/run.sh @@ -40,7 +40,7 @@ function run() { # pre execute some ddls run_sql_file $CUR/data/pre_ddl.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists test.finish_mark_for_ddl ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark_for_ddl ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 # pause and resume changefeed makes sure changefeed sending bootstrap events # when it is resumed, so the data after pause can be decoded correctly @@ -50,11 +50,11 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql_file $CUR/data/data_gbk.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/kafka_simple_basic_avro/run.sh b/tests/integration_tests/kafka_simple_basic_avro/run.sh index ef1fe1e38cb..2322cb2d2a3 100644 --- a/tests/integration_tests/kafka_simple_basic_avro/run.sh +++ b/tests/integration_tests/kafka_simple_basic_avro/run.sh @@ -38,11 +38,11 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql_file $CUR/data/data_gbk.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/kafka_simple_claim_check/run.sh b/tests/integration_tests/kafka_simple_claim_check/run.sh index 4fc1f089d18..399b301ed54 100644 --- a/tests/integration_tests/kafka_simple_claim_check/run.sh +++ b/tests/integration_tests/kafka_simple_claim_check/run.sh @@ -48,7 +48,7 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/kafka_simple_claim_check_avro/run.sh b/tests/integration_tests/kafka_simple_claim_check_avro/run.sh index 8466b83188e..d4f3d1c444f 100644 --- a/tests/integration_tests/kafka_simple_claim_check_avro/run.sh +++ b/tests/integration_tests/kafka_simple_claim_check_avro/run.sh @@ -48,7 +48,7 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/kafka_simple_handle_key_only/run.sh b/tests/integration_tests/kafka_simple_handle_key_only/run.sh index 5d15bb9df68..538b57b418b 100644 --- a/tests/integration_tests/kafka_simple_handle_key_only/run.sh +++ b/tests/integration_tests/kafka_simple_handle_key_only/run.sh @@ -46,7 +46,7 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/kafka_simple_handle_key_only_avro/run.sh b/tests/integration_tests/kafka_simple_handle_key_only_avro/run.sh index ec475d821bc..e31099aeb59 100644 --- a/tests/integration_tests/kafka_simple_handle_key_only_avro/run.sh +++ b/tests/integration_tests/kafka_simple_handle_key_only_avro/run.sh @@ -46,7 +46,7 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/kafka_sink_error_resume/run.sh b/tests/integration_tests/kafka_sink_error_resume/run.sh index 82c15e30b99..6626bc1d178 100755 --- a/tests/integration_tests/kafka_sink_error_resume/run.sh +++ b/tests/integration_tests/kafka_sink_error_resume/run.sh @@ -42,8 +42,8 @@ function run() { cdc cli changefeed resume --changefeed-id=$changefeed_id --pd=$pd_addr ensure $MAX_RETRIES check_changefeed_status 127.0.0.1:8300 $changefeed_id "normal" - check_table_exists "kafka_sink_error_resume.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "kafka_sink_error_resume.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "kafka_sink_error_resume.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "kafka_sink_error_resume.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql "INSERT INTO kafka_sink_error_resume.t1 VALUES (),();" ${UP_TIDB_HOST} ${UP_TIDB_PORT} diff --git a/tests/integration_tests/kill_owner_with_ddl/run.sh b/tests/integration_tests/kill_owner_with_ddl/run.sh index 39475021a06..31d92aeab8f 100755 --- a/tests/integration_tests/kill_owner_with_ddl/run.sh +++ b/tests/integration_tests/kill_owner_with_ddl/run.sh @@ -54,7 +54,7 @@ function run() { cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" run_sql "CREATE DATABASE kill_owner_with_ddl;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE table kill_owner_with_ddl.t1 (id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "kill_owner_with_ddl.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "kill_owner_with_ddl.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/ddlsink/mysql/MySQLSinkExecDDLDelay=return(true);github.com/pingcap/tiflow/cdc/capture/ownerFlushIntervalInject=return(10)' kill_cdc_and_restart $pd_addr $WORK_DIR $CDC_BINARY @@ -78,7 +78,7 @@ function run() { kill_cdc_and_restart $pd_addr $WORK_DIR $CDC_BINARY for i in $(seq 1 3); do - check_table_exists "kill_owner_with_ddl.t$i" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "kill_owner_with_ddl.t$i" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/many_pk_or_uk/run.sh b/tests/integration_tests/many_pk_or_uk/run.sh index 2f1ce23d9e1..b45bb702979 100755 --- a/tests/integration_tests/many_pk_or_uk/run.sh +++ b/tests/integration_tests/many_pk_or_uk/run.sh @@ -51,7 +51,7 @@ if [ "$SINK_TYPE" != "storage" ]; then cd "$(dirname "$0")" set -o pipefail GO111MODULE=on go run main.go -config ./config.toml 2>&1 | tee $WORK_DIR/tester.log - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/diff_config.toml cleanup_process $CDC_BINARY check_logs $WORK_DIR diff --git a/tests/integration_tests/move_table/run.sh b/tests/integration_tests/move_table/run.sh index 381b4153191..e379eb20562 100644 --- a/tests/integration_tests/move_table/run.sh +++ b/tests/integration_tests/move_table/run.sh @@ -45,13 +45,13 @@ function run() { # Add a check table to reduce check time, or if we check data with sync diff # directly, there maybe a lot of diff data at first because of the incremental scan run_sql "CREATE table move_table.check1(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "move_table.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "move_table.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} cd $CUR GO111MODULE=on go run main.go 2>&1 | tee $WORK_DIR/tester.log cd $WORK_DIR - check_table_exists "move_table.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists "move_table.check1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql "truncate table move_table.usertable" ${UP_TIDB_HOST} ${UP_TIDB_PORT} # move back @@ -60,7 +60,7 @@ function run() { cd $WORK_DIR check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql "CREATE table move_table.check2(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "move_table.check2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists "move_table.check2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/mq_sink_dispatcher/run.sh b/tests/integration_tests/mq_sink_dispatcher/run.sh index 6c234a6e5d9..7f51a076f11 100644 --- a/tests/integration_tests/mq_sink_dispatcher/run.sh +++ b/tests/integration_tests/mq_sink_dispatcher/run.sh @@ -65,7 +65,7 @@ function run() { run_sql "CREATE TABLE test.finish_mark (a int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/multi_capture/run.sh b/tests/integration_tests/multi_capture/run.sh index b780fe67e3a..10366ee6c4f 100755 --- a/tests/integration_tests/multi_capture/run.sh +++ b/tests/integration_tests/multi_capture/run.sh @@ -52,7 +52,7 @@ function run() { # check tables are created and data is synchronized for i in $(seq $DB_COUNT); do - check_table_exists "multi_capture_$i.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "multi_capture_$i.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/multi_cdc_cluster/run.sh b/tests/integration_tests/multi_cdc_cluster/run.sh index 8b17118462f..a487bb3f4fb 100644 --- a/tests/integration_tests/multi_cdc_cluster/run.sh +++ b/tests/integration_tests/multi_cdc_cluster/run.sh @@ -45,8 +45,8 @@ function run() { run_sql "INSERT INTO test.multi_cdc2(id, val) VALUES (2, 2);" run_sql "INSERT INTO test.multi_cdc2(id, val) VALUES (3, 3);" - check_table_exists "test.multi_cdc1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "test.multi_cdc2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test.multi_cdc1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "test.multi_cdc2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY } diff --git a/tests/integration_tests/multi_rocks/run.sh b/tests/integration_tests/multi_rocks/run.sh index 1f919e42cc8..90439d5099a 100644 --- a/tests/integration_tests/multi_rocks/run.sh +++ b/tests/integration_tests/multi_rocks/run.sh @@ -50,7 +50,7 @@ function run() { run_sql "drop table multi_rocks.a4;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "drop table multi_rocks.a5;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "create table multi_rocks.finish_mark (id int primary key); " ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists multi_rocks.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists multi_rocks.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/multi_source/run.sh b/tests/integration_tests/multi_source/run.sh index f3017fe579e..ffa8337166d 100755 --- a/tests/integration_tests/multi_source/run.sh +++ b/tests/integration_tests/multi_source/run.sh @@ -52,13 +52,13 @@ if [ "$SINK_TYPE" != "storage" ]; then cd "$(dirname "$0")" set -o pipefail GO111MODULE=on go run main.go -config ./config.toml 2>&1 | tee $WORK_DIR/tester.log - check_table_exists mark.finish_mark_0 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_3 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_4 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_5 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists mark.finish_mark_0 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_3 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_4 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_5 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 check_sync_diff $WORK_DIR $CUR/diff_config.toml cleanup_process $CDC_BINARY check_logs $WORK_DIR @@ -74,13 +74,13 @@ if [ "$SINK_TYPE" != "storage" ]; then cd "$(dirname "$0")" set -o pipefail GO111MODULE=on go run main.go -config ./config.toml 2>&1 | tee $WORK_DIR/tester.log - check_table_exists mark.finish_mark_0 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_3 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_4 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark_5 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists mark.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists mark.finish_mark_0 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_3 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_4 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark_5 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists mark.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 check_sync_diff $WORK_DIR $CUR/diff_config.toml cleanup_process $CDC_BINARY check_logs $WORK_DIR diff --git a/tests/integration_tests/multi_tables_ddl/run.sh b/tests/integration_tests/multi_tables_ddl/run.sh index fd9952b8fcf..361ef94a213 100755 --- a/tests/integration_tests/multi_tables_ddl/run.sh +++ b/tests/integration_tests/multi_tables_ddl/run.sh @@ -93,12 +93,12 @@ function run() { esac run_sql_file $CUR/data/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t55 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t66 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t7 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t88 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists multi_tables_ddl_test.t55 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists multi_tables_ddl_test.t66 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists multi_tables_ddl_test.t7 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists multi_tables_ddl_test.t88 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists multi_tables_ddl_test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists multi_tables_ddl_test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} echo "check table exists success" # changefeed test-error will not report an error, "multi_tables_ddl_test.t555 to multi_tables_ddl_test.t55" part will be skipped. diff --git a/tests/integration_tests/multi_tables_ddl_v2/run.sh b/tests/integration_tests/multi_tables_ddl_v2/run.sh index 09fd07d89c6..e998a6955fa 100755 --- a/tests/integration_tests/multi_tables_ddl_v2/run.sh +++ b/tests/integration_tests/multi_tables_ddl_v2/run.sh @@ -58,12 +58,12 @@ function run() { fi run_sql_file $CUR/data/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t55 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t66 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t7 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists multi_tables_ddl_test.t88 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists multi_tables_ddl_test.t55 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists multi_tables_ddl_test.t66 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists multi_tables_ddl_test.t7 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists multi_tables_ddl_test.t88 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists multi_tables_ddl_test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists multi_tables_ddl_test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} echo "check table exists success" # changefeed test-error will not report an error, "multi_tables_ddl_test.t555 to multi_tables_ddl_test.t55" patr will be skipped. diff --git a/tests/integration_tests/multi_topics/run.sh b/tests/integration_tests/multi_topics/run.sh index 99f8acf6458..88f3758663c 100644 --- a/tests/integration_tests/multi_topics/run.sh +++ b/tests/integration_tests/multi_topics/run.sh @@ -40,13 +40,13 @@ function run() { # sync_diff can't check non-exist table, so we check expected tables are created in downstream first for i in $(seq 1 3); do - check_table_exists test.table${i} ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists test.table${i} ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 done check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 300 run_sql "rename table test.table1 to test.table10, test.table2 to test.table20" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists test.table10 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists test.table20 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists test.table10 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists test.table20 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 300 run_sql_file $CUR/data/step2.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} @@ -55,7 +55,7 @@ function run() { run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/test_table20?protocol=canal-json&version=${KAFKA_VERSION}&enable-tidb-extension=true" "" 20 run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/test_finish?protocol=canal-json&version=${KAFKA_VERSION}&enable-tidb-extension=true" "" "finish" - check_table_exists test.finish ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists test.finish ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 300 cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/multi_topics_v2/run.sh b/tests/integration_tests/multi_topics_v2/run.sh index 615e577e900..d35be20e004 100644 --- a/tests/integration_tests/multi_topics_v2/run.sh +++ b/tests/integration_tests/multi_topics_v2/run.sh @@ -39,13 +39,13 @@ function run() { # sync_diff can't check non-exist table, so we check expected tables are created in downstream first for i in $(seq 1 3); do - check_table_exists test.table${i} ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists test.table${i} ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 done check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 300 run_sql "rename table test.table1 to test.table10, test.table2 to test.table20" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists test.table10 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 - check_table_exists test.table20 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists test.table10 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 + check_table_exists test.table20 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 300 run_sql_file $CUR/data/step2.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} @@ -54,7 +54,7 @@ function run() { run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/test_table20?protocol=canal-json&version=${KAFKA_VERSION}&enable-tidb-extension=true" "" 20 run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/test_finish?protocol=canal-json&version=${KAFKA_VERSION}&enable-tidb-extension=true" "" "finish" - check_table_exists test.finish ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + check_table_exists test.finish ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 300 cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/new_ci_collation/run.sh b/tests/integration_tests/new_ci_collation/run.sh index a63f4810551..1c409d34409 100755 --- a/tests/integration_tests/new_ci_collation/run.sh +++ b/tests/integration_tests/new_ci_collation/run.sh @@ -51,7 +51,7 @@ function run() { run_sql_file $CUR/data/test1.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} for i in $(seq 1 5); do table="new_ci_collation_test.t$i" - check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_sql_file $CUR/data/test2.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} diff --git a/tests/integration_tests/open_protocol_claim_check/run.sh b/tests/integration_tests/open_protocol_claim_check/run.sh index eaccdea6f68..d786ea0191e 100644 --- a/tests/integration_tests/open_protocol_claim_check/run.sh +++ b/tests/integration_tests/open_protocol_claim_check/run.sh @@ -40,7 +40,7 @@ function run() { cdc_kafka_consumer --upstream-uri $SINK_URI --downstream-uri="mysql://root@127.0.0.1:3306/?safe-mode=true&batch-dml-enable=false" --upstream-tidb-dsn="root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/?" --config="$CUR/conf/changefeed.toml" 2>&1 & # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/open_protocol_handle_key_only/run.sh b/tests/integration_tests/open_protocol_handle_key_only/run.sh index 6be9ec9c250..ccc56d1936c 100644 --- a/tests/integration_tests/open_protocol_handle_key_only/run.sh +++ b/tests/integration_tests/open_protocol_handle_key_only/run.sh @@ -36,7 +36,7 @@ function run() { run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/owner_remove_table_error/run.sh b/tests/integration_tests/owner_remove_table_error/run.sh index 22d5a21c8fa..74226d77d20 100644 --- a/tests/integration_tests/owner_remove_table_error/run.sh +++ b/tests/integration_tests/owner_remove_table_error/run.sh @@ -37,7 +37,7 @@ function run() { run_sql "INSERT INTO owner_remove_table_error.t2 VALUES (),(),();" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE table owner_remove_table_error.finished_mark(id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "owner_remove_table_error.finished_mark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "owner_remove_table_error.finished_mark" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/owner_resign/run.sh b/tests/integration_tests/owner_resign/run.sh index 437caf8ce01..e3112c93fff 100755 --- a/tests/integration_tests/owner_resign/run.sh +++ b/tests/integration_tests/owner_resign/run.sh @@ -44,7 +44,7 @@ function run() { # wait table t1 is processed by cdc server ensure 10 "cdc cli processor list --server http://127.0.0.1:8301 |jq '.|length'|grep -E '^1$'" # check the t1 is replicated to downstream to make sure the t1 is dispatched to cdc1 - check_table_exists "owner_resign.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "owner_resign.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_sql "INSERT INTO owner_resign.t1 (id, val) values (1, 1);" check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/partition_table/run.sh b/tests/integration_tests/partition_table/run.sh index 4c581d4e4d9..405d1327319 100644 --- a/tests/integration_tests/partition_table/run.sh +++ b/tests/integration_tests/partition_table/run.sh @@ -17,7 +17,7 @@ function run() { # record tso before we create tables to skip the system table DDLs start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) - run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "set @@global.tidb_enable_exchange_partition=on" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY @@ -39,10 +39,10 @@ function run() { esac run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists partition_table.t ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists partition_table.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists partition_table.t2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists partition_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists partition_table.t ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists partition_table.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists partition_table.t2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists partition_table.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml diff --git a/tests/integration_tests/processor_err_chan/run.sh b/tests/integration_tests/processor_err_chan/run.sh index 318bc09d15c..b333bbc5821 100644 --- a/tests/integration_tests/processor_err_chan/run.sh +++ b/tests/integration_tests/processor_err_chan/run.sh @@ -26,10 +26,10 @@ function run() { esac run_sql "CREATE DATABASE processor_err_chan;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "CREATE DATABASE processor_err_chan;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "CREATE DATABASE processor_err_chan;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} for i in $(seq 1 10); do run_sql "CREATE table processor_err_chan.t$i (id int primary key auto_increment)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "CREATE table processor_err_chan.t$i (id int primary key auto_increment)" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "CREATE table processor_err_chan.t$i (id int primary key auto_increment)" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/ProcessorAddTableError=1*return(true)' diff --git a/tests/integration_tests/processor_etcd_worker_delay/run.sh b/tests/integration_tests/processor_etcd_worker_delay/run.sh index 45da3db2d6e..2e9da179817 100644 --- a/tests/integration_tests/processor_etcd_worker_delay/run.sh +++ b/tests/integration_tests/processor_etcd_worker_delay/run.sh @@ -49,7 +49,7 @@ function run() { run_sql "INSERT INTO processor_delay.t$i VALUES (),(),();" ${UP_TIDB_HOST} ${UP_TIDB_PORT} done for i in {1..50}; do - check_table_exists "processor_delay.t$i" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "processor_delay.t$i" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} done if grep -Fa "[PANIC]" ${WORK_DIR}/cdc.log; then diff --git a/tests/integration_tests/processor_resolved_ts_fallback/run.sh b/tests/integration_tests/processor_resolved_ts_fallback/run.sh index 1d7de3d2efb..72fb4a1e770 100755 --- a/tests/integration_tests/processor_resolved_ts_fallback/run.sh +++ b/tests/integration_tests/processor_resolved_ts_fallback/run.sh @@ -43,14 +43,14 @@ function run() { # wait table t1 is processed by cdc server ensure 10 "cdc cli processor list --server http://127.0.0.1:8301 |jq '.|length'|grep -E '^1$'" # check the t1 is replicated to downstream to make sure the t1 is dispatched to cdc1 - check_table_exists "processor_resolved_ts_fallback.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "processor_resolved_ts_fallback.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} export GO_FAILPOINTS='' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "2" --addr "127.0.0.1:8302" --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" run_sql "CREATE table processor_resolved_ts_fallback.t2(id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE table processor_resolved_ts_fallback.t3(id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "processor_resolved_ts_fallback.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "processor_resolved_ts_fallback.t3" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "processor_resolved_ts_fallback.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "processor_resolved_ts_fallback.t3" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} ensure 10 "cdc cli processor list --server http://127.0.0.1:8301 |jq '.|length'|grep -E '^2$'" run_sql "INSERT INTO processor_resolved_ts_fallback.t1 values (),(),();" ${UP_TIDB_HOST} ${UP_TIDB_PORT} diff --git a/tests/integration_tests/processor_stop_delay/run.sh b/tests/integration_tests/processor_stop_delay/run.sh index 74279bd2934..8dd911c1f1b 100644 --- a/tests/integration_tests/processor_stop_delay/run.sh +++ b/tests/integration_tests/processor_stop_delay/run.sh @@ -38,7 +38,7 @@ function run() { run_sql "CREATE DATABASE processor_stop_delay;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE table processor_stop_delay.t (id int primary key auto_increment, t datetime DEFAULT CURRENT_TIMESTAMP)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO processor_stop_delay.t values (),(),(),(),(),(),()" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "processor_stop_delay.t" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "processor_stop_delay.t" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml # pause changefeed first, and then resume the changefeed. The processor stop diff --git a/tests/integration_tests/region_merge/run.sh b/tests/integration_tests/region_merge/run.sh index ed297be9aca..48a73655081 100644 --- a/tests/integration_tests/region_merge/run.sh +++ b/tests/integration_tests/region_merge/run.sh @@ -72,7 +72,7 @@ EOF done run_sql "insert into region_merge.t1 values (-9223372036854775808),(0),(1),(9223372036854775807);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists region_merge.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists region_merge.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/resolve_lock/run.sh b/tests/integration_tests/resolve_lock/run.sh index d919986d175..c355c2a2823 100755 --- a/tests/integration_tests/resolve_lock/run.sh +++ b/tests/integration_tests/resolve_lock/run.sh @@ -45,7 +45,7 @@ prepare $* cd "$(dirname "$0")" set -o pipefail GO111MODULE=on go run main.go -config ./config.toml 2>&1 | tee $WORK_DIR/tester.log -check_table_exists test.t2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} +check_table_exists test.t2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/diff_config.toml cleanup_process $CDC_BINARY check_logs $WORK_DIR diff --git a/tests/integration_tests/resourcecontrol/run.sh b/tests/integration_tests/resourcecontrol/run.sh index 2cd22f5713e..f5186699784 100644 --- a/tests/integration_tests/resourcecontrol/run.sh +++ b/tests/integration_tests/resourcecontrol/run.sh @@ -38,7 +38,7 @@ function run() { esac run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists resourcecontrol.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists resourcecontrol.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/row_format/run.sh b/tests/integration_tests/row_format/run.sh index 3a10df26ba4..69f03f42929 100644 --- a/tests/integration_tests/row_format/run.sh +++ b/tests/integration_tests/row_format/run.sh @@ -54,7 +54,7 @@ function run() { run_sql_file $CUR/data/step4.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists row_format.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists row_format.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/savepoint/run.sh b/tests/integration_tests/savepoint/run.sh index f5605af023d..c2619ec584e 100644 --- a/tests/integration_tests/savepoint/run.sh +++ b/tests/integration_tests/savepoint/run.sh @@ -38,7 +38,7 @@ function run() { esac run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists savepoint.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists savepoint.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/sequence/run.sh b/tests/integration_tests/sequence/run.sh index 3bc45696b37..12f5f1eac34 100755 --- a/tests/integration_tests/sequence/run.sh +++ b/tests/integration_tests/sequence/run.sh @@ -30,18 +30,18 @@ function run() { run_sql_file $CUR/data/test.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists sequence_test.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists sequence_test.t1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} echo "check table exists success" check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 60 # TiCDC discards all SEQUENCE DDL for now. # See https://github.com/pingcap/tiflow/issues/4559 - ! run_sql "SHOW CREATE SEQUENCE sequence_test.seq0;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + ! run_sql "SHOW CREATE SEQUENCE sequence_test.seq0;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_sql "DROP SEQUENCE sequence_test.seq0;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} # Make sure changefeed is normal. run_sql "CREATE table sequence_test.mark_table(id int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "sequence_test.mark_table" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 90 + check_table_exists "sequence_test.mark_table" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 90 cleanup_process $CDC_BINARY } diff --git a/tests/integration_tests/server_config_compatibility/run.sh b/tests/integration_tests/server_config_compatibility/run.sh index f2fcdb7feb5..f349ab3156a 100644 --- a/tests/integration_tests/server_config_compatibility/run.sh +++ b/tests/integration_tests/server_config_compatibility/run.sh @@ -32,7 +32,7 @@ function sql_check() { # the following statement will be not executed # check table simple1. - run_sql "SELECT id, val FROM test.simple1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "SELECT id, val FROM test.simple1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "id: 1" && check_contains "val: 1" && check_contains "id: 2" && @@ -40,7 +40,7 @@ function sql_check() { check_not_contains "id: 3" && # check table simple2. - run_sql "SELECT id, val FROM test.simple2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "SELECT id, val FROM test.simple2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "id: 1" && check_contains "val: 1" && check_contains "id: 2" && diff --git a/tests/integration_tests/simple/run.sh b/tests/integration_tests/simple/run.sh index 2773977d55d..06b087c5b5d 100644 --- a/tests/integration_tests/simple/run.sh +++ b/tests/integration_tests/simple/run.sh @@ -46,7 +46,7 @@ function sql_check() { # the following statement will be not executed # check table simple1. - run_sql "SELECT id, val FROM test.simple1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "SELECT id, val FROM test.simple1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "id: 1" && check_contains "val: 1" && check_contains "id: 2" && @@ -54,7 +54,7 @@ function sql_check() { check_not_contains "id: 3" && # check table simple2. - run_sql "SELECT id, val FROM test.simple2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "SELECT id, val FROM test.simple2;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "id: 1" && check_contains "val: 1" && check_contains "id: 2" && diff --git a/tests/integration_tests/sink_hang/run.sh b/tests/integration_tests/sink_hang/run.sh index ce8f172a4e3..27fdd9d9609 100644 --- a/tests/integration_tests/sink_hang/run.sh +++ b/tests/integration_tests/sink_hang/run.sh @@ -51,8 +51,8 @@ function run() { ensure $MAX_RETRIES check_changefeed_status 127.0.0.1:8300 $changefeed_id "normal" "last_error" "null" ensure $MAX_RETRIES check_changefeed_status 127.0.0.1:8300 $changefeed_id "normal" "last_warning" "null" - check_table_exists "sink_hang.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists "sink_hang.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "sink_hang.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "sink_hang.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/sink_retry/run.sh b/tests/integration_tests/sink_retry/run.sh index 0bc7cf49013..083eece7552 100755 --- a/tests/integration_tests/sink_retry/run.sh +++ b/tests/integration_tests/sink_retry/run.sh @@ -43,13 +43,13 @@ function run() { run_sql "CREATE TABLE sink_retry.finish_mark_1 (a int primary key);" sleep 30 - check_table_exists "sink_retry.finish_mark_1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + check_table_exists "sink_retry.finish_mark_1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 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=sink_retry run_sql "CREATE TABLE sink_retry.finish_mark_2 (a int primary key);" sleep 30 - check_table_exists "sink_retry.finish_mark_2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + check_table_exists "sink_retry.finish_mark_2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY diff --git a/tests/integration_tests/split_region/run.sh b/tests/integration_tests/split_region/run.sh index 301b7fc0218..d4c5401a7bd 100755 --- a/tests/integration_tests/split_region/run.sh +++ b/tests/integration_tests/split_region/run.sh @@ -52,8 +52,8 @@ EOF esac # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists split_region.test1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - check_table_exists split_region.test2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists split_region.test1 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists split_region.test2 ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml # split table into 5 regions, run some other DMLs and check data is synchronized to downstream diff --git a/tests/integration_tests/sql_mode/run.sh b/tests/integration_tests/sql_mode/run.sh index 394ca9b4e8d..8e30d30a538 100644 --- a/tests/integration_tests/sql_mode/run.sh +++ b/tests/integration_tests/sql_mode/run.sh @@ -39,7 +39,7 @@ start_tidb_cluster --workdir $WORK_DIR trap stop_tidb_cluster EXIT run_sql "set global sql_mode='NO_BACKSLASH_ESCAPES';" ${UP_TIDB_HOST} ${UP_TIDB_PORT} -run_sql "set global sql_mode='NO_BACKSLASH_ESCAPES';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} +run_sql "set global sql_mode='NO_BACKSLASH_ESCAPES';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} cd $WORK_DIR start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) @@ -51,9 +51,9 @@ run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --chan run_sql "use test; create table t1(id bigint primary key, a text, b text as ((regexp_replace(a, '^[1-9]\d{9,29}$', 'aaaaa'))), c text); insert into t1 (id, a, c) values(1,123456, 'ab\\\\\\\\c'); insert into t1 (id, a, c) values(2,1234567890123, 'ab\\\\c');" ${UP_TIDB_HOST} ${UP_TIDB_PORT} if [ "$SINK_TYPE" == "mysql" ]; then - check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sleep 10 - run_sql "SELECT * from test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "SELECT * from test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "b: 123456" && check_contains "b: aaaaa" && check_contains "c: ab\\\\\\\\c" && @@ -65,7 +65,7 @@ start_tidb_cluster --workdir $WORK_DIR ## case 2 run_sql "set global sql_mode='ANSI_QUOTES';" ${UP_TIDB_HOST} ${UP_TIDB_PORT} -run_sql "set global sql_mode='ANSI_QUOTES';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} +run_sql "set global sql_mode='ANSI_QUOTES';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY @@ -76,9 +76,9 @@ run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --chan run_sql "use test; create table t2(id bigint primary key, a date); insert into t2 values(1, '2023-02-08');" ${UP_TIDB_HOST} ${UP_TIDB_PORT} if [ "$SINK_TYPE" == "mysql" ]; then - check_table_exists "test.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sleep 10 - run_sql "SELECT * from test.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} && + run_sql "SELECT * from test.t2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} && check_contains "a: 2023-02-08" fi diff --git a/tests/integration_tests/synced_status/run.sh b/tests/integration_tests/synced_status/run.sh index 7bbe0d42649..042c65e839e 100644 --- a/tests/integration_tests/synced_status/run.sh +++ b/tests/integration_tests/synced_status/run.sh @@ -95,7 +95,7 @@ function run_normal_case_and_unavailable_pd() { fi run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" - check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sleep 5 # wait data insert synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) @@ -147,7 +147,7 @@ function run_case_with_unavailable_tikv() { # case 3: test in unavailable tikv cluster run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" - check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sleep 5 # make data inserted into downstream kill_tikv @@ -209,7 +209,7 @@ function run_case_with_unavailable_tidb() { # case 3: test in unavailable tikv cluster run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" - check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sleep 5 # make data inserted into downstream kill_tidb diff --git a/tests/integration_tests/synced_status_with_redo/run.sh b/tests/integration_tests/synced_status_with_redo/run.sh index 9e5fb22a98b..b1255aa631c 100644 --- a/tests/integration_tests/synced_status_with_redo/run.sh +++ b/tests/integration_tests/synced_status_with_redo/run.sh @@ -99,7 +99,7 @@ function run_normal_case_and_unavailable_pd() { fi run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" - check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sleep 5 # wait data insert synced_status=$(curl -X GET http://127.0.0.1:8300/api/v2/changefeeds/test-1/synced) @@ -151,7 +151,7 @@ function run_case_with_unavailable_tikv() { # case 3: test in unavailable tikv cluster run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" - check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sleep 5 # make data inserted into downstream kill_tikv @@ -213,7 +213,7 @@ function run_case_with_unavailable_tidb() { # case 3: test in unavailable tikv cluster run_sql "USE TEST;Create table t1(a int primary key, b int);insert into t1 values(1,2);insert into t1 values(2,3);" - check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists "test.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sleep 5 # make data inserted into downstream kill_tidb diff --git a/tests/integration_tests/syncpoint/run.sh b/tests/integration_tests/syncpoint/run.sh index 9098a381a5f..3546b88faf1 100755 --- a/tests/integration_tests/syncpoint/run.sh +++ b/tests/integration_tests/syncpoint/run.sh @@ -152,9 +152,9 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - run_sql "SET GLOBAL tidb_enable_external_ts_read = on;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "SET GLOBAL tidb_enable_external_ts_read = on;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_sql "CREATE DATABASE testSync;" - run_sql "CREATE DATABASE testSync;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "CREATE DATABASE testSync;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} # make suer no panic happen when the syncpoint enable and the ddl sink initializing slowly export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/DDLSinkInitializeSlowly=return(true)' @@ -168,14 +168,14 @@ function run() { 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} + check_table_exists "testSync.usertable" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "testSync.simple1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + check_table_exists "testSync.simple2" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} - run_sql "SET GLOBAL tidb_enable_external_ts_read = off;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "SET GLOBAL tidb_enable_external_ts_read = off;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sleep 60 - run_sql "SELECT primary_ts, secondary_ts FROM tidb_cdc.syncpoint_v1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "SELECT primary_ts, secondary_ts FROM tidb_cdc.syncpoint_v1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} echo "____________________________________" cat "$OUT_DIR/sql_res.$TEST_NAME.txt" checkDiff diff --git a/tests/integration_tests/syncpoint_check_ts/run.sh b/tests/integration_tests/syncpoint_check_ts/run.sh index f6fb07cf932..7cd3d06756d 100644 --- a/tests/integration_tests/syncpoint_check_ts/run.sh +++ b/tests/integration_tests/syncpoint_check_ts/run.sh @@ -29,7 +29,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - run_sql "SET GLOBAL tidb_enable_external_ts_read = on;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "SET GLOBAL tidb_enable_external_ts_read = on;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY @@ -39,10 +39,10 @@ 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" --config="$CUR/conf/changefeed.toml" - run_sql "SET GLOBAL tidb_enable_external_ts_read = off;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "SET GLOBAL tidb_enable_external_ts_read = off;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} sleep 60 - run_sql "SELECT primary_ts, secondary_ts FROM tidb_cdc.syncpoint_v1 order by primary_ts limit 1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + run_sql "SELECT primary_ts, secondary_ts FROM tidb_cdc.syncpoint_v1 order by primary_ts limit 1;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} echo "____________________________________" cat "$OUT_DIR/sql_res.$TEST_NAME.txt" primary_ts=($(grep primary_ts $OUT_DIR/sql_res.$TEST_NAME.txt | awk -F ": " '{print $2}')) diff --git a/tests/integration_tests/tidb_mysql_test/run.sh b/tests/integration_tests/tidb_mysql_test/run.sh index 5ed2573ef3b..cf7b458d259 100755 --- a/tests/integration_tests/tidb_mysql_test/run.sh +++ b/tests/integration_tests/tidb_mysql_test/run.sh @@ -57,7 +57,7 @@ cd "$(dirname "$0")" # run mysql-test cases ./test.sh mysql -h${UP_TIDB_HOST} -P${UP_TIDB_PORT} -uroot -e "create table test.finish_mark(id int primary key)" -check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 +check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 300 check_sync_diff $WORK_DIR $CUR/diff_config.toml cleanup_process $CDC_BINARY check_logs $WORK_DIR diff --git a/tests/integration_tests/tiflash/run.sh b/tests/integration_tests/tiflash/run.sh index b344f731d44..f8e021cbc32 100644 --- a/tests/integration_tests/tiflash/run.sh +++ b/tests/integration_tests/tiflash/run.sh @@ -50,7 +50,7 @@ EOF esac run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first - check_table_exists cdc_tiflash_test.multi_data_type ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + check_table_exists cdc_tiflash_test.multi_data_type ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml cleanup_process $CDC_BINARY From ea7a1a95bb97d37291a239f1e75d15428459cf73 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 23 Aug 2024 16:32:28 +0800 Subject: [PATCH 15/50] fix(start_tidb_cluster): fix the error in shell parsing of the port --- tests/integration_tests/_utils/start_tidb_cluster_impl | 9 ++++++--- tests/integration_tests/_utils/test_prepare | 2 -- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/integration_tests/_utils/start_tidb_cluster_impl b/tests/integration_tests/_utils/start_tidb_cluster_impl index 7370b83a515..ada9423715e 100755 --- a/tests/integration_tests/_utils/start_tidb_cluster_impl +++ b/tests/integration_tests/_utils/start_tidb_cluster_impl @@ -251,13 +251,15 @@ tidb-server \ echo "Starting Downstream TiDB..." tidb-server -V for i in $(seq 1 $downstream_db); do + port_var="DOWN_TIDB_PORT_${i}" + status_var="DOWN_TIDB_STATUS_${i}" randomGenSocketsConf tidb-server \ - -P ${!DOWN_TIDB_PORT_${i}} \ + -P ${!port_var} \ -config "$random_file_name" \ --store tikv \ --path ${DOWN_PD_HOST}:${DOWN_PD_PORT} \ - --status=${!DOWN_TIDB_STATUS_${i}} \ + --status=${!status_var} \ --log-file "$OUT_DIR/tidb_down_$i.log" & done @@ -284,8 +286,9 @@ done echo "Verifying Downstream TiDB is started..." for index in $(seq 1 $downstream_db); do + port_var="DOWN_TIDB_PORT_${index}" i=0 - while ! mysql -uroot -h${DOWN_TIDB_HOST} -P${!DOWN_TIDB_PORT_${index}} --default-character-set utf8mb4 -e 'select * from mysql.tidb;'; do + while ! mysql -uroot -h${DOWN_TIDB_HOST} -P${!port_var} --default-character-set utf8mb4 -e 'select * from mysql.tidb;'; do i=$((i + 1)) if [ "$i" -gt 60 ]; then echo 'Failed to start downstream TiDB' diff --git a/tests/integration_tests/_utils/test_prepare b/tests/integration_tests/_utils/test_prepare index 1f2c897397e..c59889da0ad 100644 --- a/tests/integration_tests/_utils/test_prepare +++ b/tests/integration_tests/_utils/test_prepare @@ -4,8 +4,6 @@ UP_TIDB_OTHER_PORT=${UP_TIDB_OTHER_PORT:-4001} UP_TIDB_STATUS=${UP_TIDB_STATUS:-10080} UP_TIDB_OTHER_STATUS=${UP_TIDB_OTHER_STATUS:-10081} DOWN_TIDB_HOST=${DOWN_TIDB_HOST:-127.0.0.1} -; DOWN_TIDB_PORT=${DOWN_TIDB_PORT:-3306} -; DOWN_TIDB_STATUS=${DOWN_TIDB_STATUS:-20080} DOWN_TIDB_PORT_1=${DOWN_TIDB_PORT_1:-3306} DOWN_TIDB_STATUS_1=${DOWN_TIDB_STATUS_1:-20080} DOWN_TIDB_PORT_2=${DOWN_TIDB_PORT_2:-3307} From 9cc3c491e753cd579c19c1529d6c75c084682d80 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 23 Aug 2024 16:47:15 +0800 Subject: [PATCH 16/50] fix(test_prepare): fix the error of parsing DOWN_TIDB_STATUS by modifying all DOWN_TIDB_STATUS to DOWN_TIDB_STATUS_1 --- tests/integration_tests/_utils/stop_tidb_cluster | 2 +- tests/integration_tests/bank/run.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration_tests/_utils/stop_tidb_cluster b/tests/integration_tests/_utils/stop_tidb_cluster index 8d9dbd7998a..1d903e06c2a 100755 --- a/tests/integration_tests/_utils/stop_tidb_cluster +++ b/tests/integration_tests/_utils/stop_tidb_cluster @@ -27,7 +27,7 @@ kill -9 $(lsof -i tcp:${UP_TIDB_OTHER_PORT} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_TIDB_STATUS} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_TIDB_OTHER_STATUS} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${DOWN_TIDB_PORT_1} -t 2>/dev/null) &>/dev/null || true -kill -9 $(lsof -i tcp:${DOWN_TIDB_STATUS} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${DOWN_TIDB_STATUS_1} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_PD_PORT_1} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_PD_PEER_PORT_1} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_PD_PORT_2} -t 2>/dev/null) &>/dev/null || true diff --git a/tests/integration_tests/bank/run.sh b/tests/integration_tests/bank/run.sh index 10e5e85226b..6f585ad0aca 100644 --- a/tests/integration_tests/bank/run.sh +++ b/tests/integration_tests/bank/run.sh @@ -34,7 +34,7 @@ if [ "$SINK_TYPE" == "mysql" ]; then GO111MODULE=on go run bank.go case.go -u "root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/bank" \ -d "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1})/bank" --test-round=20000 \ - -a "${DOWN_TIDB_HOST}:${DOWN_TIDB_STATUS}" + -a "${DOWN_TIDB_HOST}:${DOWN_TIDB_STATUS_1}" cleanup_process $CDC_BINARY echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" From a772191da64db9eac63ddff8373ae769c0237948 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 23 Aug 2024 17:19:01 +0800 Subject: [PATCH 17/50] feat(start_tidb_cluster): add `--downstream_db` option to the `start_tidb_cluster` command --- tests/integration_tests/_utils/start_tidb_cluster | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration_tests/_utils/start_tidb_cluster b/tests/integration_tests/_utils/start_tidb_cluster index e7b214b7649..ed1d874f244 100755 --- a/tests/integration_tests/_utils/start_tidb_cluster +++ b/tests/integration_tests/_utils/start_tidb_cluster @@ -65,13 +65,13 @@ while [ $i -le $retry_times ]; do echo "The ${i} times to try to start tidb cluster..." if [[ "$tidb_config" != "" ]]; then - start_tidb_cluster_impl --workdir ${OUT_DIR} --multiple-upstream-pd ${multiple_upstream_pd} --tidb-config ${tidb_config} + start_tidb_cluster_impl --workdir ${OUT_DIR} --multiple-upstream-pd ${multiple_upstream_pd} --downstream_db ${downstream_db} --tidb-config ${tidb_config} elif [[ "$pd_config" != "" ]]; then - start_tidb_cluster_impl --workdir ${OUT_DIR} --multiple-upstream-pd ${multiple_upstream_pd} --pd-config ${pd_config} + start_tidb_cluster_impl --workdir ${OUT_DIR} --multiple-upstream-pd ${multiple_upstream_pd} --downstream_db ${downstream_db} --pd-config ${pd_config} elif [[ "$tikv_config" != "" ]]; then - start_tidb_cluster_impl --workdir ${OUT_DIR} --multiple-upstream-pd ${multiple_upstream_pd} --tikv-config ${tikv_config} + start_tidb_cluster_impl --workdir ${OUT_DIR} --multiple-upstream-pd ${multiple_upstream_pd} --downstream_db ${downstream_db} --tikv-config ${tikv_config} else - start_tidb_cluster_impl --workdir ${OUT_DIR} --multiple-upstream-pd ${multiple_upstream_pd} + start_tidb_cluster_impl --workdir ${OUT_DIR} --multiple-upstream-pd ${multiple_upstream_pd} --downstream_db ${downstream_db} fi if [ $? -eq 0 ]; then From 199b14ca7da2e3f6b92dca559909881e8a690c65 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 23 Aug 2024 18:27:04 +0800 Subject: [PATCH 18/50] feat(start tidb): extract the code for starting the downstream tidb instances into a separate script --- .../_utils/start_downstream_tidb_instances | 84 +++++++++++++++++++ .../_utils/start_tidb_cluster_impl | 39 ++------- 2 files changed, 91 insertions(+), 32 deletions(-) create mode 100644 tests/integration_tests/_utils/start_downstream_tidb_instances diff --git a/tests/integration_tests/_utils/start_downstream_tidb_instances b/tests/integration_tests/_utils/start_downstream_tidb_instances new file mode 100644 index 00000000000..4d10ac7e0c1 --- /dev/null +++ b/tests/integration_tests/_utils/start_downstream_tidb_instances @@ -0,0 +1,84 @@ +#!/bin/bash + +# Input: db, out_dir +# --db: the number of tidb instances +# --out_dir: the directory to store the log files of tidb instances +db= +out_dir= + +random_file_name= + +while [[ ${1} ]]; do + case "${1}" in + --db) + db=${2} + shift + ;; + --workdir) + out_dir=${2} + shift + ;; + *) + echo "Unknown parameter: ${1}" >&2 + exit 1 + ;; + esac + + if ! shift; then + echo 'Missing parameter argument.' >&2 + exit 1 + fi +done + +# Ensure out_dir exists +mkdir -p "$out_dir" + +# Function to generate random socket config +randomGenSocketsConf() { + random_str=$(date '+%s%N') + + # For macOS, generate a random string from /dev/random + if [ "$(uname)" == "Darwin" ]; then + random_str=$(cat /dev/random | LC_ALL=C tr -dc "a-zA-Z0-9" | head -c 10) + fi + + # Create a unique configuration file + random_file_name="$out_dir/tidb-config-$random_str.toml" + + # Copy base configuration and append socket configuration + cat "$out_dir/tidb-config.toml" >"$random_file_name" + echo "socket = \"/tmp/tidb-$random_str.sock\"" >>"$random_file_name" +} + +echo "Starting Downstream TiDB..." +tidb-server -V +for i in $(seq 1 $db); do + port_var="DOWN_TIDB_PORT_${i}" + status_var="DOWN_TIDB_STATUS_${i}" + + # Generate random socket configuration + randomGenSocketsConf + + # Start the tidb-server + tidb-server \ + -P ${!port_var} \ + -config "$random_file_name" \ + --store tikv \ + --path ${DOWN_PD_HOST}:${DOWN_PD_PORT} \ + --status=${!status_var} \ + --log-file "$out_dir/tidb_down_$i.log" & +done + +echo "Verifying Downstream TiDB is started..." +for index in $(seq 1 $db); do + port_var="DOWN_TIDB_PORT_${index}" + i=0 + while ! mysql -uroot -h${DOWN_TIDB_HOST} -P${!port_var} --default-character-set utf8mb4 -e 'select * from mysql.tidb;'; do + i=$((i + 1)) + if [ "$i" -gt 60 ]; then + echo 'Failed to start downstream TiDB' + exit 1 + fi + sleep 2 + done +done diff --git a/tests/integration_tests/_utils/start_tidb_cluster_impl b/tests/integration_tests/_utils/start_tidb_cluster_impl index ada9423715e..dd9be72ee6f 100755 --- a/tests/integration_tests/_utils/start_tidb_cluster_impl +++ b/tests/integration_tests/_utils/start_tidb_cluster_impl @@ -248,21 +248,6 @@ tidb-server \ --status=${UP_TIDB_OTHER_STATUS} \ --log-file "$OUT_DIR/tidb_other.log" & -echo "Starting Downstream TiDB..." -tidb-server -V -for i in $(seq 1 $downstream_db); do - port_var="DOWN_TIDB_PORT_${i}" - status_var="DOWN_TIDB_STATUS_${i}" - randomGenSocketsConf - tidb-server \ - -P ${!port_var} \ - -config "$random_file_name" \ - --store tikv \ - --path ${DOWN_PD_HOST}:${DOWN_PD_PORT} \ - --status=${!status_var} \ - --log-file "$OUT_DIR/tidb_down_$i.log" & -done - echo "Verifying Upstream TiDB is started..." i=0 while ! mysql -uroot -h${UP_TIDB_HOST} -P${UP_TIDB_PORT} --default-character-set utf8mb4 -e 'select * from mysql.tidb;'; do @@ -284,25 +269,15 @@ while ! mysql -uroot -h${UP_TIDB_HOST} -P${UP_TIDB_OTHER_PORT} --default-charact sleep 2 done -echo "Verifying Downstream TiDB is started..." -for index in $(seq 1 $downstream_db); do - port_var="DOWN_TIDB_PORT_${index}" - i=0 - while ! mysql -uroot -h${DOWN_TIDB_HOST} -P${!port_var} --default-character-set utf8mb4 -e 'select * from mysql.tidb;'; do - i=$((i + 1)) - if [ "$i" -gt 60 ]; then - echo 'Failed to start downstream TiDB' - exit 1 - fi - sleep 2 - done -done +start_downstream_tidb_instances --db ${downstream_db} --out_dir ${OUT_DIR} run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${UP_TIDB_HOST} ${UP_TIDB_PORT} -run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} -run_sql "CREATE user 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} -run_sql "GRANT select,insert,update,delete,index,create,drop,alter,create view,references ON *.* TO 'normal'@'%';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} -run_sql "FLUSH privileges" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} +if [ "$downstream_db" -gt 0]; then + run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + run_sql "CREATE user 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + run_sql "GRANT select,insert,update,delete,index,create,drop,alter,create view,references ON *.* TO 'normal'@'%';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} + run_sql "FLUSH privileges" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} +fi cat - >"$OUT_DIR/tiflash-config.toml" < Date: Fri, 23 Aug 2024 18:40:47 +0800 Subject: [PATCH 19/50] chore(start tidb): modify `start_downstream_tidb_instances` to executable --- tests/integration_tests/_utils/start_downstream_tidb_instances | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/integration_tests/_utils/start_downstream_tidb_instances diff --git a/tests/integration_tests/_utils/start_downstream_tidb_instances b/tests/integration_tests/_utils/start_downstream_tidb_instances old mode 100644 new mode 100755 From 829c6f307efe06d3bb64f7d913340050c0af6655 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 23 Aug 2024 18:44:12 +0800 Subject: [PATCH 20/50] fix(start tidb): fix the typo "workdir" --- tests/integration_tests/_utils/start_downstream_tidb_instances | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/_utils/start_downstream_tidb_instances b/tests/integration_tests/_utils/start_downstream_tidb_instances index 4d10ac7e0c1..4f9dfd3a160 100755 --- a/tests/integration_tests/_utils/start_downstream_tidb_instances +++ b/tests/integration_tests/_utils/start_downstream_tidb_instances @@ -14,7 +14,7 @@ while [[ ${1} ]]; do db=${2} shift ;; - --workdir) + --out_dir) out_dir=${2} shift ;; From e1967b03738f17582de9c33d1e1ac324de982a42 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 23 Aug 2024 19:07:19 +0800 Subject: [PATCH 21/50] fix(start tidb): add `source test_prepare` to `start_downstream_tidb_instances` --- tests/integration_tests/_utils/start_downstream_tidb_instances | 3 +++ tests/integration_tests/_utils/start_tidb_cluster_impl | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration_tests/_utils/start_downstream_tidb_instances b/tests/integration_tests/_utils/start_downstream_tidb_instances index 4f9dfd3a160..ff2db940782 100755 --- a/tests/integration_tests/_utils/start_downstream_tidb_instances +++ b/tests/integration_tests/_utils/start_downstream_tidb_instances @@ -30,6 +30,9 @@ while [[ ${1} ]]; do fi done +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare + # Ensure out_dir exists mkdir -p "$out_dir" diff --git a/tests/integration_tests/_utils/start_tidb_cluster_impl b/tests/integration_tests/_utils/start_tidb_cluster_impl index dd9be72ee6f..80853e0f0f5 100755 --- a/tests/integration_tests/_utils/start_tidb_cluster_impl +++ b/tests/integration_tests/_utils/start_tidb_cluster_impl @@ -272,7 +272,7 @@ done start_downstream_tidb_instances --db ${downstream_db} --out_dir ${OUT_DIR} run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${UP_TIDB_HOST} ${UP_TIDB_PORT} -if [ "$downstream_db" -gt 0]; then +if [ "$downstream_db" -gt 0 ]; then run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_sql "CREATE user 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} run_sql "GRANT select,insert,update,delete,index,create,drop,alter,create view,references ON *.* TO 'normal'@'%';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} From 36770a0748e67ec74ce30a6bb0f146b7e2138b62 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 23 Aug 2024 19:09:48 +0800 Subject: [PATCH 22/50] feat(stop_tidb_cluster): add the operation to stop downstream tidb 2 and 3 --- tests/integration_tests/_utils/stop_tidb_cluster | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/integration_tests/_utils/stop_tidb_cluster b/tests/integration_tests/_utils/stop_tidb_cluster index 1d903e06c2a..76fa850649b 100755 --- a/tests/integration_tests/_utils/stop_tidb_cluster +++ b/tests/integration_tests/_utils/stop_tidb_cluster @@ -28,6 +28,10 @@ kill -9 $(lsof -i tcp:${UP_TIDB_STATUS} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_TIDB_OTHER_STATUS} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${DOWN_TIDB_PORT_1} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${DOWN_TIDB_STATUS_1} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${DOWN_TIDB_PORT_2} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${DOWN_TIDB_STATUS_2} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${DOWN_TIDB_PORT_3} -t 2>/dev/null) &>/dev/null || true +kill -9 $(lsof -i tcp:${DOWN_TIDB_STATUS_3} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_PD_PORT_1} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_PD_PEER_PORT_1} -t 2>/dev/null) &>/dev/null || true kill -9 $(lsof -i tcp:${UP_PD_PORT_2} -t 2>/dev/null) &>/dev/null || true From b2c11e4e273ce98bbb5c71ca1593c1d1d0cc93bb Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 23 Aug 2024 19:15:41 +0800 Subject: [PATCH 23/50] feat(start tidb): add a check for the value of the `--db` parameter --- .../integration_tests/_utils/start_downstream_tidb_instances | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration_tests/_utils/start_downstream_tidb_instances b/tests/integration_tests/_utils/start_downstream_tidb_instances index ff2db940782..5a53545431f 100755 --- a/tests/integration_tests/_utils/start_downstream_tidb_instances +++ b/tests/integration_tests/_utils/start_downstream_tidb_instances @@ -30,6 +30,11 @@ while [[ ${1} ]]; do fi done +if [ "$db" -gt 3 ]; then + echo 'The integration test only supports a maximum of 3 TiDB instances' + exit 1 +fi + CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) source $CUR/../_utils/test_prepare From 207f3950ae9d00e01f1c5062dc3290dcd9495079 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Mon, 26 Aug 2024 16:24:20 +0800 Subject: [PATCH 24/50] feat(start tidb): support start tidb instances with the specific port --- .../_utils/start_downstream_tidb_instances | 30 ++++++++++++++----- .../_utils/start_tidb_cluster_impl | 6 ---- 2 files changed, 23 insertions(+), 13 deletions(-) diff --git a/tests/integration_tests/_utils/start_downstream_tidb_instances b/tests/integration_tests/_utils/start_downstream_tidb_instances index 5a53545431f..5ca4729200a 100755 --- a/tests/integration_tests/_utils/start_downstream_tidb_instances +++ b/tests/integration_tests/_utils/start_downstream_tidb_instances @@ -1,10 +1,14 @@ #!/bin/bash -# Input: db, out_dir -# --db: the number of tidb instances -# --out_dir: the directory to store the log files of tidb instances +# Input: db, out_dir, suffix +# --db: The number of tidb instances +# --out_dir: The directory to store the log files of tidb instances +# --suffix: Refers to the suffix of `DOWN_TIDB_PORT` as defined in the `test_prepare` file. +# NOTE: This is helpful when using multiple downstream TiDB instances. +# If only one downstream TiDB instance is needed, this can be ignored. db= out_dir= +suffix=1 random_file_name= @@ -18,6 +22,10 @@ while [[ ${1} ]]; do out_dir=${2} shift ;; + --suffix) + suffix=${2} + shift + ;; *) echo "Unknown parameter: ${1}" >&2 exit 1 @@ -30,8 +38,8 @@ while [[ ${1} ]]; do fi done -if [ "$db" -gt 3 ]; then - echo 'The integration test only supports a maximum of 3 TiDB instances' +if [ $((db + suffix - 1)) -gt 3 ]; then + echo 'The integration test only supports a maximum of 3 downstream TiDB instances' exit 1 fi @@ -60,7 +68,7 @@ randomGenSocketsConf() { echo "Starting Downstream TiDB..." tidb-server -V -for i in $(seq 1 $db); do +for i in $(seq $suffix $((db + suffix - 1))); do port_var="DOWN_TIDB_PORT_${i}" status_var="DOWN_TIDB_STATUS_${i}" @@ -78,7 +86,7 @@ for i in $(seq 1 $db); do done echo "Verifying Downstream TiDB is started..." -for index in $(seq 1 $db); do +for index in $(seq $suffix $((db + suffix - 1))); do port_var="DOWN_TIDB_PORT_${index}" i=0 while ! mysql -uroot -h${DOWN_TIDB_HOST} -P${!port_var} --default-character-set utf8mb4 -e 'select * from mysql.tidb;'; do @@ -90,3 +98,11 @@ for index in $(seq 1 $db); do sleep 2 done done + +if [ "$db" -gt 0 ]; then + port_var="DOWN_TIDB_PORT_${suffix}" + run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${DOWN_TIDB_HOST} ${!port_var} + run_sql "CREATE user 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${!port_var} + run_sql "GRANT select,insert,update,delete,index,create,drop,alter,create view,references ON *.* TO 'normal'@'%';" ${DOWN_TIDB_HOST} ${!port_var} + run_sql "FLUSH privileges" ${DOWN_TIDB_HOST} ${!port_var} +fi diff --git a/tests/integration_tests/_utils/start_tidb_cluster_impl b/tests/integration_tests/_utils/start_tidb_cluster_impl index 80853e0f0f5..67000df4602 100755 --- a/tests/integration_tests/_utils/start_tidb_cluster_impl +++ b/tests/integration_tests/_utils/start_tidb_cluster_impl @@ -272,12 +272,6 @@ done start_downstream_tidb_instances --db ${downstream_db} --out_dir ${OUT_DIR} run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${UP_TIDB_HOST} ${UP_TIDB_PORT} -if [ "$downstream_db" -gt 0 ]; then - run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} - run_sql "CREATE user 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} - run_sql "GRANT select,insert,update,delete,index,create,drop,alter,create view,references ON *.* TO 'normal'@'%';" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} - run_sql "FLUSH privileges" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} -fi cat - >"$OUT_DIR/tiflash-config.toml" < Date: Mon, 26 Aug 2024 23:22:33 +0800 Subject: [PATCH 25/50] feat(start tidb): capture the PIDs of TiDB instances started by start_downstream_tidb_instances script --- tests/integration_tests/_utils/start_downstream_tidb_instances | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration_tests/_utils/start_downstream_tidb_instances b/tests/integration_tests/_utils/start_downstream_tidb_instances index 5ca4729200a..cd96dac0250 100755 --- a/tests/integration_tests/_utils/start_downstream_tidb_instances +++ b/tests/integration_tests/_utils/start_downstream_tidb_instances @@ -48,6 +48,7 @@ source $CUR/../_utils/test_prepare # Ensure out_dir exists mkdir -p "$out_dir" +rm -f "$out_dir/downstream_tidb_instances_pids.log" # Function to generate random socket config randomGenSocketsConf() { @@ -83,6 +84,7 @@ for i in $(seq $suffix $((db + suffix - 1))); do --path ${DOWN_PD_HOST}:${DOWN_PD_PORT} \ --status=${!status_var} \ --log-file "$out_dir/tidb_down_$i.log" & + echo $! >> "$out_dir/downstream_tidb_instances_pids.log" done echo "Verifying Downstream TiDB is started..." From fb712a5d5487bed1741901065a3ce428971701ea Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 27 Aug 2024 15:40:27 +0800 Subject: [PATCH 26/50] test(multi addresses): add a integration test for the multi downstream addresses feature --- .../conf/diff_config_1.toml | 29 +++++ .../conf/diff_config_2.toml | 29 +++++ .../conf/diff_config_3.toml | 29 +++++ .../multi_down_addresses/data/prepare.sql | 3 + .../multi_down_addresses/run.sh | 105 ++++++++++++++++++ 5 files changed, 195 insertions(+) create mode 100644 tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml create mode 100644 tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml create mode 100644 tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml create mode 100644 tests/integration_tests/multi_down_addresses/data/prepare.sql create mode 100644 tests/integration_tests/multi_down_addresses/run.sh diff --git a/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml b/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml new file mode 100644 index 00000000000..a7e02f2fa16 --- /dev/null +++ b/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/multi_down_addresses/sync_diff/output" + + source-instances = ["up"] + + target-instance = "down" + + target-check-tables = ["test.round"] + +[data-sources] +[data-sources.up] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + +[data-sources.down] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" diff --git a/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml b/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml new file mode 100644 index 00000000000..1c7e399e59d --- /dev/null +++ b/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/multi_down_addresses/sync_diff/output" + + source-instances = ["up"] + + target-instance = "down" + + target-check-tables = ["test.round"] + +[data-sources] +[data-sources.up] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + +[data-sources.down] + host = "127.0.0.1" + port = 3307 + user = "root" + password = "" diff --git a/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml b/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml new file mode 100644 index 00000000000..a3d59928e51 --- /dev/null +++ b/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml @@ -0,0 +1,29 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/multi_down_addresses/sync_diff/output" + + source-instances = ["up"] + + target-instance = "down" + + target-check-tables = ["test.round"] + +[data-sources] +[data-sources.up] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + +[data-sources.down] + host = "127.0.0.1" + port = 3308 + user = "root" + password = "" diff --git a/tests/integration_tests/multi_down_addresses/data/prepare.sql b/tests/integration_tests/multi_down_addresses/data/prepare.sql new file mode 100644 index 00000000000..d6d1e9c0553 --- /dev/null +++ b/tests/integration_tests/multi_down_addresses/data/prepare.sql @@ -0,0 +1,3 @@ +drop database if exists test; +create database test; +use test; diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh new file mode 100644 index 00000000000..167d0c1970a --- /dev/null +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -0,0 +1,105 @@ +#!/bin/bash + +set -eu + +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 + +function prepare() { + # Start upstream and downstream TiDB cluster + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + start_tidb_cluster --workdir $WORK_DIR --downstream_db 0 + cd $WORK_DIR + run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + + # Start downstream TiDB instances + start_downstream_tidb_instances --db 3 --out_dir $WORK_DIR + mapfile -t down_tidb_pids < "$WORK_DIR/downstream_tidb_instances_pids.log" + echo "Started downstream TiDB instances with PIDs: ${down_tidb_pids[@]}" + + # Start the CDC synchronization task. + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + run_cdc_cli changefeed create --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_2},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_3}/" + sleep 5 + + # Prepare a table + run_sql "CREATE TABLE round VALUES(round INT, val INT, success BOOLEAN);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists test.round ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 +} + +function run() { + pid1 = ${down_tidb_pids[0]} + pid2 = ${down_tidb_pids[1]} + pid3 = ${down_tidb_pids[2]} + + # Round 1 + # shutdown tidb 1 -> begin -> insert -> commit -> check_sync_diff + # tidb 2 should works + kill $pid1 + run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO round VALUES(1, 1, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/diff_config_2.toml + + # Round 2 + # begin -> shutdown tidb 2 -> insert -> commit -> check_sync_diff + # tidb 3 should works + run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + sleep 5 + kill $pid2 + run_sql "INSERT INTO round VALUES(2, 1, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/diff_config_3.toml + + # Round 3 + # begin -> insert -> recover tidb 1 -> shutdown tidb 3 -> commit -> check_sync_diff + # tidb 1 should works + run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO round VALUES(3, 1, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + sleep 5 + start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 + mapfile -t down_tidb_pids < "$WORK_DIR/downstream_tidb_instances_pids.log" + pid1 = ${down_tidb_pids[0]} + kill $pid3 + run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/diff_config_1.toml + + # Round 4 + # begin -> insert -> recover tidb 2 -> shutdown tidb 1 -> insert -> commit -> check_sync_diff + # tidb 2 should works + run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO round VALUES(4, 1, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + sleep 5 + start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 + mapfile -t down_tidb_pids < "$WORK_DIR/downstream_tidb_instances_pids.log" + pid2 = ${down_tidb_pids[0]} + kill $pid1 + run_sql "INSERT INTO round VALUES(4, 2, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/diff_config_2.toml + + # Round 5 + # begin -> insert -> commit -> recover tidb 3 -> shutdown tidb 2 -> begin -> insert -> commit -> check_sync_diff + # tidb 3 should works + run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO round VALUES(5, 1, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + sleep 5 + start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 3 + kill $pid2 + run_sql "INSERT INTO round VALUES(5, 2, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/diff_config_3.toml +} + +# No need to support kafka and storage sink. +if [ "$SINK_TYPE" == "mysql" ]; then + trap stop_tidb_cluster EXIT + prepare $* + run $* + check_logs $WORK_DIR + echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" +fi From e7808a60efa7eedffe248c2153b0c030c84843b1 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 27 Aug 2024 18:21:32 +0800 Subject: [PATCH 27/50] fix(connector): fix a data race in `(*mysqlBackend).Close()` --- cdc/sink/dmlsink/txn/mysql/mysql.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index f03c9cf79ec..a3c10eeeb9f 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -61,7 +61,7 @@ const ( type mysqlBackend struct { workerID int changefeed string - connector *pmysql.MySQLDBConnector + connector pmysql.MySQLDBConnector cfg *pmysql.Config dmlMaxRetry uint64 @@ -177,7 +177,7 @@ func NewMySQLBackends( backends = append(backends, &mysqlBackend{ workerID: i, changefeed: changefeed, - connector: connector, + connector: *connector, cfg: cfg, dmlMaxRetry: defaultDMLMaxRetry, statistics: statistics, From b8b6c49015f8da578273a728f3b84d5cbd48e369 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 27 Aug 2024 18:29:56 +0800 Subject: [PATCH 28/50] fix(multi_down_addresses): fix the wrong SQL in integration test multi_down_addresses --- .../conf/diff_config_1.toml | 2 +- .../conf/diff_config_2.toml | 2 +- .../conf/diff_config_3.toml | 2 +- .../multi_down_addresses/data/prepare.sql | 8 +++-- .../multi_down_addresses/run.sh | 29 +++++++++---------- 5 files changed, 22 insertions(+), 21 deletions(-) diff --git a/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml b/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml index a7e02f2fa16..c94b682a3dd 100644 --- a/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml +++ b/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml @@ -13,7 +13,7 @@ check-struct-only = false target-instance = "down" - target-check-tables = ["test.round"] + target-check-tables = ["multi_down_addresses.round"] [data-sources] [data-sources.up] diff --git a/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml b/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml index 1c7e399e59d..27ed498ee7c 100644 --- a/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml +++ b/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml @@ -13,7 +13,7 @@ check-struct-only = false target-instance = "down" - target-check-tables = ["test.round"] + target-check-tables = ["multi_down_addresses.round"] [data-sources] [data-sources.up] diff --git a/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml b/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml index a3d59928e51..542b91ac42d 100644 --- a/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml +++ b/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml @@ -13,7 +13,7 @@ check-struct-only = false target-instance = "down" - target-check-tables = ["test.round"] + target-check-tables = ["multi_down_addresses.round"] [data-sources] [data-sources.up] diff --git a/tests/integration_tests/multi_down_addresses/data/prepare.sql b/tests/integration_tests/multi_down_addresses/data/prepare.sql index d6d1e9c0553..845c4571833 100644 --- a/tests/integration_tests/multi_down_addresses/data/prepare.sql +++ b/tests/integration_tests/multi_down_addresses/data/prepare.sql @@ -1,3 +1,5 @@ -drop database if exists test; -create database test; -use test; +drop database if exists multi_down_addresses; +create database multi_down_addresses; +use multi_down_addresses; + +CREATE TABLE round (round INT PRIMARY KEY, val INT); diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh index 167d0c1970a..7e851e39534 100644 --- a/tests/integration_tests/multi_down_addresses/run.sh +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -13,7 +13,6 @@ function prepare() { rm -rf $WORK_DIR && mkdir -p $WORK_DIR start_tidb_cluster --workdir $WORK_DIR --downstream_db 0 cd $WORK_DIR - run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} # Start downstream TiDB instances start_downstream_tidb_instances --db 3 --out_dir $WORK_DIR @@ -26,21 +25,21 @@ function prepare() { sleep 5 # Prepare a table - run_sql "CREATE TABLE round VALUES(round INT, val INT, success BOOLEAN);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists test.round ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 + run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists multi_down_addresses.round ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 } function run() { - pid1 = ${down_tidb_pids[0]} - pid2 = ${down_tidb_pids[1]} - pid3 = ${down_tidb_pids[2]} + pid1=${down_tidb_pids[0]} + pid2=${down_tidb_pids[1]} + pid3=${down_tidb_pids[2]} # Round 1 # shutdown tidb 1 -> begin -> insert -> commit -> check_sync_diff # tidb 2 should works kill $pid1 run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO round VALUES(1, 1, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.round VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/diff_config_2.toml @@ -50,7 +49,7 @@ function run() { run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 kill $pid2 - run_sql "INSERT INTO round VALUES(2, 1, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.round VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/diff_config_3.toml @@ -58,11 +57,11 @@ function run() { # begin -> insert -> recover tidb 1 -> shutdown tidb 3 -> commit -> check_sync_diff # tidb 1 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO round VALUES(3, 1, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.round VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 mapfile -t down_tidb_pids < "$WORK_DIR/downstream_tidb_instances_pids.log" - pid1 = ${down_tidb_pids[0]} + pid1=${down_tidb_pids[0]} kill $pid3 run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/diff_config_1.toml @@ -71,13 +70,13 @@ function run() { # begin -> insert -> recover tidb 2 -> shutdown tidb 1 -> insert -> commit -> check_sync_diff # tidb 2 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO round VALUES(4, 1, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.round VALUES(4, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 mapfile -t down_tidb_pids < "$WORK_DIR/downstream_tidb_instances_pids.log" - pid2 = ${down_tidb_pids[0]} + pid2=${down_tidb_pids[0]} kill $pid1 - run_sql "INSERT INTO round VALUES(4, 2, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.round VALUES(4, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/diff_config_2.toml @@ -85,12 +84,12 @@ function run() { # begin -> insert -> commit -> recover tidb 3 -> shutdown tidb 2 -> begin -> insert -> commit -> check_sync_diff # tidb 3 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO round VALUES(5, 1, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.round VALUES(5, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 3 kill $pid2 - run_sql "INSERT INTO round VALUES(5, 2, true);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.round VALUES(5, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/diff_config_3.toml } From 3304d217fdb143b1ca0833442b420922e1bf2129 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 27 Aug 2024 18:32:35 +0800 Subject: [PATCH 29/50] fix(multi_down_addresses): fix the wrong config path for check_sync_diff --- tests/integration_tests/multi_down_addresses/run.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh index 7e851e39534..f3c96a5c1d6 100644 --- a/tests/integration_tests/multi_down_addresses/run.sh +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -41,7 +41,7 @@ function run() { run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO multi_down_addresses.round VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_sync_diff $WORK_DIR $CUR/diff_config_2.toml + check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml # Round 2 # begin -> shutdown tidb 2 -> insert -> commit -> check_sync_diff @@ -51,7 +51,7 @@ function run() { kill $pid2 run_sql "INSERT INTO multi_down_addresses.round VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_sync_diff $WORK_DIR $CUR/diff_config_3.toml + check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml # Round 3 # begin -> insert -> recover tidb 1 -> shutdown tidb 3 -> commit -> check_sync_diff @@ -64,7 +64,7 @@ function run() { pid1=${down_tidb_pids[0]} kill $pid3 run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_sync_diff $WORK_DIR $CUR/diff_config_1.toml + check_sync_diff $WORK_DIR $CUR/conf/diff_config_1.toml # Round 4 # begin -> insert -> recover tidb 2 -> shutdown tidb 1 -> insert -> commit -> check_sync_diff @@ -78,7 +78,7 @@ function run() { kill $pid1 run_sql "INSERT INTO multi_down_addresses.round VALUES(4, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_sync_diff $WORK_DIR $CUR/diff_config_2.toml + check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml # Round 5 # begin -> insert -> commit -> recover tidb 3 -> shutdown tidb 2 -> begin -> insert -> commit -> check_sync_diff @@ -91,7 +91,7 @@ function run() { kill $pid2 run_sql "INSERT INTO multi_down_addresses.round VALUES(5, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_sync_diff $WORK_DIR $CUR/diff_config_3.toml + check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml } # No need to support kafka and storage sink. From fd7d71c6044c4819e00ea9c76c4f5e1bf24255dd Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 27 Aug 2024 18:38:40 +0800 Subject: [PATCH 30/50] fix(multi_down_addresses): fix the wrong SQL of duplicate entry for a primary key error --- tests/integration_tests/multi_down_addresses/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh index f3c96a5c1d6..edf1a762d03 100644 --- a/tests/integration_tests/multi_down_addresses/run.sh +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -76,7 +76,7 @@ function run() { mapfile -t down_tidb_pids < "$WORK_DIR/downstream_tidb_instances_pids.log" pid2=${down_tidb_pids[0]} kill $pid1 - run_sql "INSERT INTO multi_down_addresses.round VALUES(4, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.round VALUES(41, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml @@ -89,7 +89,7 @@ function run() { sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 3 kill $pid2 - run_sql "INSERT INTO multi_down_addresses.round VALUES(5, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.round VALUES(51, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml } From dd4f0744bfa13c230116635b9d423252e0b3e09a Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 27 Aug 2024 18:45:38 +0800 Subject: [PATCH 31/50] fix(start_downstream_tidb_instances): fix the idempotence of `CREATE USER` --- tests/integration_tests/_utils/start_downstream_tidb_instances | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/_utils/start_downstream_tidb_instances b/tests/integration_tests/_utils/start_downstream_tidb_instances index cd96dac0250..2c29f75540a 100755 --- a/tests/integration_tests/_utils/start_downstream_tidb_instances +++ b/tests/integration_tests/_utils/start_downstream_tidb_instances @@ -104,7 +104,7 @@ done if [ "$db" -gt 0 ]; then port_var="DOWN_TIDB_PORT_${suffix}" run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${DOWN_TIDB_HOST} ${!port_var} - run_sql "CREATE user 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${!port_var} + run_sql "CREATE USER IF NOT EXISTS 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${!port_var} run_sql "GRANT select,insert,update,delete,index,create,drop,alter,create view,references ON *.* TO 'normal'@'%';" ${DOWN_TIDB_HOST} ${!port_var} run_sql "FLUSH privileges" ${DOWN_TIDB_HOST} ${!port_var} fi From 50573e964096c8c3329c6672b341d8eb7b076316 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Tue, 27 Aug 2024 19:22:39 +0800 Subject: [PATCH 32/50] chore(connector): add the copyright information for the new files --- pkg/sink/mysql/mysql_connector.go | 13 +++++++++++++ pkg/sink/mysql/mysql_connector_test.go | 13 +++++++++++++ 2 files changed, 26 insertions(+) diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go index 8f442f7271b..c02c6768e82 100644 --- a/pkg/sink/mysql/mysql_connector.go +++ b/pkg/sink/mysql/mysql_connector.go @@ -1,3 +1,16 @@ +// Copyright 2024 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 mysql import ( diff --git a/pkg/sink/mysql/mysql_connector_test.go b/pkg/sink/mysql/mysql_connector_test.go index ea8043fdc49..fff05064538 100644 --- a/pkg/sink/mysql/mysql_connector_test.go +++ b/pkg/sink/mysql/mysql_connector_test.go @@ -1,3 +1,16 @@ +// Copyright 2024 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 mysql import ( From b7c112a272ae9384f684de55df7403622cd64a96 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 28 Aug 2024 11:14:04 +0800 Subject: [PATCH 33/50] chore(integration test): add the `multi_down_addresses` integration test to the `run_group.sh` --- tests/integration_tests/run_group.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/run_group.sh b/tests/integration_tests/run_group.sh index a1ce80ead0a..79a8c651717 100755 --- a/tests/integration_tests/run_group.sh +++ b/tests/integration_tests/run_group.sh @@ -42,7 +42,7 @@ groups=( # G04 'foreign_key ddl_puller_lag ddl_only_block_related_table changefeed_auto_stop' # G05 - 'charset_gbk ddl_manager multi_source' # + 'charset_gbk ddl_manager multi_source multi_down_addresses' # G06 'sink_retry changefeed_error ddl_sequence resourcecontrol' # G07 pulsar oauth2 authentication enabled From 72e6e316f9892061028c145a25163bda4bb3951f Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 28 Aug 2024 11:33:07 +0800 Subject: [PATCH 34/50] chore(integration test): format the new files by `make check` --- .../_utils/start_downstream_tidb_instances | 88 +++++++++---------- .../multi_down_addresses/run.sh | 6 +- 2 files changed, 47 insertions(+), 47 deletions(-) diff --git a/tests/integration_tests/_utils/start_downstream_tidb_instances b/tests/integration_tests/_utils/start_downstream_tidb_instances index 2c29f75540a..c393961cd60 100755 --- a/tests/integration_tests/_utils/start_downstream_tidb_instances +++ b/tests/integration_tests/_utils/start_downstream_tidb_instances @@ -4,7 +4,7 @@ # --db: The number of tidb instances # --out_dir: The directory to store the log files of tidb instances # --suffix: Refers to the suffix of `DOWN_TIDB_PORT` as defined in the `test_prepare` file. -# NOTE: This is helpful when using multiple downstream TiDB instances. +# NOTE: This is helpful when using multiple downstream TiDB instances. # If only one downstream TiDB instance is needed, this can be ignored. db= out_dir= @@ -18,11 +18,11 @@ while [[ ${1} ]]; do db=${2} shift ;; - --out_dir) + --out_dir) out_dir=${2} shift ;; - --suffix) + --suffix) suffix=${2} shift ;; @@ -39,8 +39,8 @@ while [[ ${1} ]]; do done if [ $((db + suffix - 1)) -gt 3 ]; then - echo 'The integration test only supports a maximum of 3 downstream TiDB instances' - exit 1 + echo 'The integration test only supports a maximum of 3 downstream TiDB instances' + exit 1 fi CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -52,57 +52,57 @@ rm -f "$out_dir/downstream_tidb_instances_pids.log" # Function to generate random socket config randomGenSocketsConf() { - random_str=$(date '+%s%N') - - # For macOS, generate a random string from /dev/random - if [ "$(uname)" == "Darwin" ]; then - random_str=$(cat /dev/random | LC_ALL=C tr -dc "a-zA-Z0-9" | head -c 10) - fi - - # Create a unique configuration file - random_file_name="$out_dir/tidb-config-$random_str.toml" - - # Copy base configuration and append socket configuration - cat "$out_dir/tidb-config.toml" >"$random_file_name" - echo "socket = \"/tmp/tidb-$random_str.sock\"" >>"$random_file_name" + random_str=$(date '+%s%N') + + # For macOS, generate a random string from /dev/random + if [ "$(uname)" == "Darwin" ]; then + random_str=$(cat /dev/random | LC_ALL=C tr -dc "a-zA-Z0-9" | head -c 10) + fi + + # Create a unique configuration file + random_file_name="$out_dir/tidb-config-$random_str.toml" + + # Copy base configuration and append socket configuration + cat "$out_dir/tidb-config.toml" >"$random_file_name" + echo "socket = \"/tmp/tidb-$random_str.sock\"" >>"$random_file_name" } echo "Starting Downstream TiDB..." tidb-server -V for i in $(seq $suffix $((db + suffix - 1))); do - port_var="DOWN_TIDB_PORT_${i}" - status_var="DOWN_TIDB_STATUS_${i}" - - # Generate random socket configuration - randomGenSocketsConf - - # Start the tidb-server - tidb-server \ - -P ${!port_var} \ - -config "$random_file_name" \ - --store tikv \ - --path ${DOWN_PD_HOST}:${DOWN_PD_PORT} \ - --status=${!status_var} \ - --log-file "$out_dir/tidb_down_$i.log" & - echo $! >> "$out_dir/downstream_tidb_instances_pids.log" + port_var="DOWN_TIDB_PORT_${i}" + status_var="DOWN_TIDB_STATUS_${i}" + + # Generate random socket configuration + randomGenSocketsConf + + # Start the tidb-server + tidb-server \ + -P ${!port_var} \ + -config "$random_file_name" \ + --store tikv \ + --path ${DOWN_PD_HOST}:${DOWN_PD_PORT} \ + --status=${!status_var} \ + --log-file "$out_dir/tidb_down_$i.log" & + echo $! >>"$out_dir/downstream_tidb_instances_pids.log" done echo "Verifying Downstream TiDB is started..." for index in $(seq $suffix $((db + suffix - 1))); do - port_var="DOWN_TIDB_PORT_${index}" - i=0 - while ! mysql -uroot -h${DOWN_TIDB_HOST} -P${!port_var} --default-character-set utf8mb4 -e 'select * from mysql.tidb;'; do - i=$((i + 1)) - if [ "$i" -gt 60 ]; then - echo 'Failed to start downstream TiDB' - exit 1 - fi - sleep 2 - done + port_var="DOWN_TIDB_PORT_${index}" + i=0 + while ! mysql -uroot -h${DOWN_TIDB_HOST} -P${!port_var} --default-character-set utf8mb4 -e 'select * from mysql.tidb;'; do + i=$((i + 1)) + if [ "$i" -gt 60 ]; then + echo 'Failed to start downstream TiDB' + exit 1 + fi + sleep 2 + done done if [ "$db" -gt 0 ]; then - port_var="DOWN_TIDB_PORT_${suffix}" + port_var="DOWN_TIDB_PORT_${suffix}" run_sql "update mysql.tidb set variable_value='60m' where variable_name='tikv_gc_life_time';" ${DOWN_TIDB_HOST} ${!port_var} run_sql "CREATE USER IF NOT EXISTS 'normal'@'%' identified by '123456';" ${DOWN_TIDB_HOST} ${!port_var} run_sql "GRANT select,insert,update,delete,index,create,drop,alter,create view,references ON *.* TO 'normal'@'%';" ${DOWN_TIDB_HOST} ${!port_var} diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh index edf1a762d03..d0d037aa859 100644 --- a/tests/integration_tests/multi_down_addresses/run.sh +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -16,7 +16,7 @@ function prepare() { # Start downstream TiDB instances start_downstream_tidb_instances --db 3 --out_dir $WORK_DIR - mapfile -t down_tidb_pids < "$WORK_DIR/downstream_tidb_instances_pids.log" + mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" echo "Started downstream TiDB instances with PIDs: ${down_tidb_pids[@]}" # Start the CDC synchronization task. @@ -60,7 +60,7 @@ function run() { run_sql "INSERT INTO multi_down_addresses.round VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 - mapfile -t down_tidb_pids < "$WORK_DIR/downstream_tidb_instances_pids.log" + mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid1=${down_tidb_pids[0]} kill $pid3 run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} @@ -73,7 +73,7 @@ function run() { run_sql "INSERT INTO multi_down_addresses.round VALUES(4, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 - mapfile -t down_tidb_pids < "$WORK_DIR/downstream_tidb_instances_pids.log" + mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid2=${down_tidb_pids[0]} kill $pid1 run_sql "INSERT INTO multi_down_addresses.round VALUES(41, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} From 4d8c0d47d5fbe8713551d3edf184bcc760517eaa Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 28 Aug 2024 11:51:37 +0800 Subject: [PATCH 35/50] docs(connector): add and modify some comments by following the rule of `make check` --- pkg/retry/options.go | 2 ++ pkg/sink/mysql/mysql_connector.go | 4 ++-- pkg/sink/mysql/mysql_connector_test.go | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/pkg/retry/options.go b/pkg/retry/options.go index fc85881fb24..5e9d7dcfb90 100644 --- a/pkg/retry/options.go +++ b/pkg/retry/options.go @@ -33,6 +33,8 @@ type Option func(*retryOptions) // IsRetryable checks the error is safe or worth to retry, eg. "context.Canceled" better not retry type IsRetryable func(error) bool +// PreExecution defines a function type that is expected to return an error. +// This function is designed to be executed before retry attempts, but only after the initial execution has failed. type PreExecution func() error type retryOptions struct { diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go index c02c6768e82..aef1d3799d5 100644 --- a/pkg/sink/mysql/mysql_connector.go +++ b/pkg/sink/mysql/mysql_connector.go @@ -50,12 +50,12 @@ type MySQLDBConnector struct { configureDBWhenSwitch func() } -// New a MySQLDBConnector for creating DB connection. +// NewMySQLDBConnector new a MySQLDBConnector for creating DB connection. func NewMySQLDBConnector(ctx context.Context, cfg *Config, sinkURI *url.URL) (*MySQLDBConnector, error) { return NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, &DBConnectionFactory{}) } -// New a MySQLDBConnector by the given factory function for creating DB connection. +// NewMySQLDBConnectorWithFactory new a MySQLDBConnector by the given factory function for creating DB connection. // The sinkURI format: // [scheme]://[user[:password]@][host[:port]][,host[:port]][,host[:port]][/path][?param1=value1¶mN=valueN] // User must ensure that each address ([host[:port]]) in the sinkURI (if there are multiple addresses) diff --git a/pkg/sink/mysql/mysql_connector_test.go b/pkg/sink/mysql/mysql_connector_test.go index fff05064538..ff047dc4ac3 100644 --- a/pkg/sink/mysql/mysql_connector_test.go +++ b/pkg/sink/mysql/mysql_connector_test.go @@ -148,7 +148,7 @@ func TestSwitchToAvailableMySQLDB_SwitchDB(t *testing.T) { require.Nil(t, err) mock.ExpectPing() mock.ExpectClose() - return db, nil + return db, err } var dbCandidates [3]*sql.DB From 44a3c9bbe74a30fec8975151cd066428e22882b8 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 28 Aug 2024 11:56:23 +0800 Subject: [PATCH 36/50] refactor(connector): rename `MySQLDBConnector` to `DBConnector` by the suggestion of `make check` --- cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 4 +-- cdc/sink/dmlsink/txn/mysql/mysql.go | 4 +-- cdc/syncpointstore/mysql_syncpoint_store.go | 4 +-- pkg/sink/mysql/mysql_connector.go | 30 ++++++++++----------- pkg/sink/mysql/mysql_connector_test.go | 26 +++++++++--------- pkg/sink/observer/observer.go | 2 +- pkg/sink/observer/tidb.go | 6 ++--- pkg/sink/observer/tidb_test.go | 2 +- 8 files changed, 39 insertions(+), 39 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index 8a2529cdfa7..6e7fbcec344 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -55,7 +55,7 @@ var _ ddlsink.Sink = (*DDLSink)(nil) type DDLSink struct { // id indicates which processor (changefeed) this sink belongs to. id model.ChangeFeedID - connector *pmysql.MySQLDBConnector + connector *pmysql.DBConnector cfg *pmysql.Config // statistics is the statistics of this sink. // We use it to record the DDL count. @@ -80,7 +80,7 @@ func NewDDLSink( return nil, err } - connector, err := pmysql.NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, GetDBConnImpl) + connector, err := pmysql.NewDBConnectorWithFactory(ctx, cfg, sinkURI, GetDBConnImpl) if err != nil { return nil, err } diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index a3c10eeeb9f..e8fe00969da 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -61,7 +61,7 @@ const ( type mysqlBackend struct { workerID int changefeed string - connector pmysql.MySQLDBConnector + connector pmysql.DBConnector cfg *pmysql.Config dmlMaxRetry uint64 @@ -97,7 +97,7 @@ func NewMySQLBackends( return nil, err } - connector, err := pmysql.NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + connector, err := pmysql.NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) if err != nil { return nil, err } diff --git a/cdc/syncpointstore/mysql_syncpoint_store.go b/cdc/syncpointstore/mysql_syncpoint_store.go index 9910d8f1012..cd46828a0a1 100644 --- a/cdc/syncpointstore/mysql_syncpoint_store.go +++ b/cdc/syncpointstore/mysql_syncpoint_store.go @@ -35,7 +35,7 @@ import ( type mysqlSyncPointStore struct { id model.ChangeFeedID cfg *pmysql.Config - connector *pmysql.MySQLDBConnector + connector *pmysql.DBConnector clusterID string syncPointRetention time.Duration lastCleanSyncPointTime time.Time @@ -54,7 +54,7 @@ func newMySQLSyncPointStore( return nil, err } - connector, err := pmysql.NewMySQLDBConnector(ctx, cfg, sinkURI) + connector, err := pmysql.NewDBConnector(ctx, cfg, sinkURI) if err != nil { return nil, err } diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go index aef1d3799d5..a536927b6c8 100644 --- a/pkg/sink/mysql/mysql_connector.go +++ b/pkg/sink/mysql/mysql_connector.go @@ -22,11 +22,11 @@ import ( "go.uber.org/zap" ) -// MySQLDBConnector manages the database connection, handling reconnections +// DBConnector manages the database connection, handling reconnections // in case of connection failures when calling SwitchToAvailableMySQLDB method. // To execute SQL queries or interact with the database, use the CurrentDB field of -// MySQLDBConnector to access the underlying *sql.DB instance, i.e., MySQLDBConnector.CurrentDB. -type MySQLDBConnector struct { +// DBConnector to access the underlying *sql.DB instance, i.e., DBConnector.CurrentDB. +type DBConnector struct { // Current connnection to a MySQL-compatible DB. CurrentDB *sql.DB @@ -39,7 +39,7 @@ type MySQLDBConnector struct { // used to try to find an available DB server in Round-Robin mode. nextTry int - // dbConnFactory is a Factory function for MySQLDBConnector that creates database connections. + // dbConnFactory is a Factory function for DBConnector that creates database connections. dbConnFactory IDBConnectionFactory // configureDBWhenSwitch stores the function that will be automatically invoked @@ -50,17 +50,17 @@ type MySQLDBConnector struct { configureDBWhenSwitch func() } -// NewMySQLDBConnector new a MySQLDBConnector for creating DB connection. -func NewMySQLDBConnector(ctx context.Context, cfg *Config, sinkURI *url.URL) (*MySQLDBConnector, error) { - return NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, &DBConnectionFactory{}) +// NewDBConnector new a DBConnector for creating DB connection. +func NewDBConnector(ctx context.Context, cfg *Config, sinkURI *url.URL) (*DBConnector, error) { + return NewDBConnectorWithFactory(ctx, cfg, sinkURI, &DBConnectionFactory{}) } -// NewMySQLDBConnectorWithFactory new a MySQLDBConnector by the given factory function for creating DB connection. +// NewDBConnectorWithFactory new a DBConnector by the given factory function for creating DB connection. // The sinkURI format: // [scheme]://[user[:password]@][host[:port]][,host[:port]][,host[:port]][/path][?param1=value1¶mN=valueN] // User must ensure that each address ([host[:port]]) in the sinkURI (if there are multiple addresses) // is valid, otherwise returns an error. -func NewMySQLDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *url.URL, dbConnFactory IDBConnectionFactory) (*MySQLDBConnector, error) { +func NewDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *url.URL, dbConnFactory IDBConnectionFactory) (*DBConnector, error) { if dbConnFactory == nil { dbConnFactory = &DBConnectionFactory{} } @@ -75,7 +75,7 @@ func NewMySQLDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *u return nil, err } - connector := &MySQLDBConnector{dsnList: dsnList, nextTry: 0, dbConnFactory: dbConnFactory} + connector := &DBConnector{dsnList: dsnList, nextTry: 0, dbConnFactory: dbConnFactory} err = connector.SwitchToAvailableMySQLDB(ctx) if err != nil { @@ -86,9 +86,9 @@ func NewMySQLDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *u } // SwitchToAvailableMySQLDB attempts to switch to an available MySQL-compatible database connection -// if the current connection is invalid, and it updates MySQLDBConnector.CurrentDB for user to use it. -// If there is only one DSN in MySQLDBConnector, switching is not possible. -func (c *MySQLDBConnector) SwitchToAvailableMySQLDB(ctx context.Context) error { +// if the current connection is invalid, and it updates DBConnector.CurrentDB for user to use it. +// If there is only one DSN in DBConnector, switching is not possible. +func (c *DBConnector) SwitchToAvailableMySQLDB(ctx context.Context) error { // If a connection has already been established and there is only one DSN (Data Source Name) available, // it is not possible to connect a different DSN. Therefore, simply return from the function. if len(c.dsnList) == 1 && c.CurrentDB != nil { @@ -134,10 +134,10 @@ func (c *MySQLDBConnector) SwitchToAvailableMySQLDB(ctx context.Context) error { // when switching to an available connection using the SwitchToAvailableMySQLDB function. // By providing the function `f` as an argument, it ensures that any newly established // connection is automatically configured after the switch. Additionally, if the existing -// MySQLDBConnector.CurrentDB also requires configuration, you can set +// DBConnector.CurrentDB also requires configuration, you can set // `needConfigureCurrentDB` to true, and this function will automatically apply // the configuration function `f` to it as well. -func (c *MySQLDBConnector) ConfigureDBWhenSwitch(f func(), needConfigureCurrentDB bool) { +func (c *DBConnector) ConfigureDBWhenSwitch(f func(), needConfigureCurrentDB bool) { if f == nil { return } diff --git a/pkg/sink/mysql/mysql_connector_test.go b/pkg/sink/mysql/mysql_connector_test.go index ff047dc4ac3..a3c8041570e 100644 --- a/pkg/sink/mysql/mysql_connector_test.go +++ b/pkg/sink/mysql/mysql_connector_test.go @@ -24,8 +24,8 @@ import ( "github.com/stretchr/testify/require" ) -// Test NewMySQLDBConnectorWithFactory for successful single address initialization -func TestNewMySQLDBConnectorWithFactory_SingleAddressSuccess(t *testing.T) { +// Test NewDBConnectorWithFactory for successful single address initialization +func TestNewDBConnectorWithFactory_SingleAddressSuccess(t *testing.T) { ctx := context.Background() sinkURI, _ := url.Parse("mysql://user:password@localhost") cfg := NewConfig() @@ -41,7 +41,7 @@ func TestNewMySQLDBConnectorWithFactory_SingleAddressSuccess(t *testing.T) { return db, nil }) - connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + connector, err := NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) require.NoError(t, err) require.NotNil(t, connector) require.NotNil(t, connector.CurrentDB) @@ -49,8 +49,8 @@ func TestNewMySQLDBConnectorWithFactory_SingleAddressSuccess(t *testing.T) { require.NoError(t, connector.CurrentDB.Close()) } -// Test NewMySQLDBConnectorWithFactory for successful multiple addresses initialization -func TestNewMySQLDBConnectorWithFactory_MultiAddressSuccess(t *testing.T) { +// Test NewDBConnectorWithFactory for successful multiple addresses initialization +func TestNewDBConnectorWithFactory_MultiAddressSuccess(t *testing.T) { ctx := context.Background() // Create a sinkURI which contains 3 addresses sinkURI, _ := url.Parse("mysql://user:password@localhost,localhost,localhost") @@ -73,7 +73,7 @@ func TestNewMySQLDBConnectorWithFactory_MultiAddressSuccess(t *testing.T) { return db, nil }) - connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + connector, err := NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) require.NoError(t, err) require.NotNil(t, connector) require.NotNil(t, connector.CurrentDB) @@ -82,8 +82,8 @@ func TestNewMySQLDBConnectorWithFactory_MultiAddressSuccess(t *testing.T) { require.NoError(t, connector.CurrentDB.Close()) } -// Test NewMySQLDBConnectorWithFactory for error when generate DSNs fail -func TestNewMySQLDBConnectorWithFactory_generateDSNsFail(t *testing.T) { +// Test NewDBConnectorWithFactory for error when generate DSNs fail +func TestNewDBConnectorWithFactory_generateDSNsFail(t *testing.T) { ctx := context.Background() sinkURI, _ := url.Parse("mysql://user:password@localhost") cfg := NewConfig() @@ -100,7 +100,7 @@ func TestNewMySQLDBConnectorWithFactory_generateDSNsFail(t *testing.T) { return nil, nil }) - connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + connector, err := NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) require.Error(t, err) require.Nil(t, connector) require.Equal(t, numCallStandard, 0) @@ -123,7 +123,7 @@ func TestSwitchToAvailableMySQLDB_CurrentDBValid(t *testing.T) { return db, nil }) - connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + connector, err := NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) require.NoError(t, err) require.NotNil(t, connector) require.NotNil(t, connector.CurrentDB) @@ -164,7 +164,7 @@ func TestSwitchToAvailableMySQLDB_SwitchDB(t *testing.T) { return db, nil }) - connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + connector, err := NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) require.NoError(t, err) require.NotNil(t, connector) require.NotNil(t, connector.CurrentDB) @@ -202,7 +202,7 @@ func TestConfigureDBWhenSwitch_Success(t *testing.T) { return db, nil }) - connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + connector, err := NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) require.NoError(t, err) require.NotNil(t, connector) require.NotNil(t, connector.CurrentDB) @@ -236,7 +236,7 @@ func TestConfigureDBWhenSwitch_NilConfigureFunction(t *testing.T) { return db, nil }) - connector, err := NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + connector, err := NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) require.NoError(t, err) require.NotNil(t, connector) require.NotNil(t, connector.CurrentDB) diff --git a/pkg/sink/observer/observer.go b/pkg/sink/observer/observer.go index fc0e505802f..356ef1070a0 100644 --- a/pkg/sink/observer/observer.go +++ b/pkg/sink/observer/observer.go @@ -80,7 +80,7 @@ func NewObserver( return nil, err } - connector, err := pmysql.NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, options.dbConnFactory) + connector, err := pmysql.NewDBConnectorWithFactory(ctx, cfg, sinkURI, options.dbConnFactory) if err != nil { return nil, err } diff --git a/pkg/sink/observer/tidb.go b/pkg/sink/observer/tidb.go index 64200d46492..ee5eb00e796 100644 --- a/pkg/sink/observer/tidb.go +++ b/pkg/sink/observer/tidb.go @@ -108,7 +108,7 @@ var ( // TiDBObserver is a tidb performance observer. It's not thread-safe. type TiDBObserver struct { - connector *mysql.MySQLDBConnector + connector *mysql.DBConnector } // Tick implements Observer @@ -174,7 +174,7 @@ func (o *TiDBObserver) Close() error { } // NewTiDBObserver creates a new TiDBObserver instance -func NewTiDBObserver(connector *mysql.MySQLDBConnector) *TiDBObserver { +func NewTiDBObserver(connector *mysql.DBConnector) *TiDBObserver { return &TiDBObserver{ connector: connector, } @@ -234,7 +234,7 @@ type metricColumnIface[T metricColumnImpl] interface { } func queryMetrics[T metricColumnImpl, F metricColumnIface[T]]( - ctx context.Context, connector *mysql.MySQLDBConnector, stmt string, metrics *[]F, + ctx context.Context, connector *mysql.DBConnector, stmt string, metrics *[]F, ) error { return retry.Do(ctx, func() error { rows, err := connector.CurrentDB.QueryContext(ctx, stmt) diff --git a/pkg/sink/observer/tidb_test.go b/pkg/sink/observer/tidb_test.go index 1e4b673bec1..0b78a04cab7 100644 --- a/pkg/sink/observer/tidb_test.go +++ b/pkg/sink/observer/tidb_test.go @@ -80,7 +80,7 @@ func TestTiDBObserver(t *testing.T) { sinkURI, err := url.Parse(sinkURIStr) require.NoError(t, err) ctx := context.Background() - connector, err := mysql.NewMySQLDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + connector, err := mysql.NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) require.NoError(t, err) observer := NewTiDBObserver(connector) From da73ffa0ba93c64e2ebd28d8fb082c885cdf0b79 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 28 Aug 2024 12:17:33 +0800 Subject: [PATCH 37/50] refactor(connector): rename the function `SwitchToAvailableMySQLDB` to `SwitchToAnAvailableDB` --- cdc/sink/ddlsink/mysql/mysql_ddl_sink.go | 2 +- cdc/sink/dmlsink/txn/mysql/mysql.go | 2 +- cdc/syncpointstore/mysql_syncpoint_store.go | 4 ++-- pkg/sink/mysql/mysql_connector.go | 12 ++++++------ pkg/sink/mysql/mysql_connector_test.go | 16 ++++++++-------- pkg/sink/observer/tidb.go | 2 +- 6 files changed, 19 insertions(+), 19 deletions(-) diff --git a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index 6e7fbcec344..dd7a7df106a 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -152,7 +152,7 @@ func (m *DDLSink) execDDLWithMaxRetries(ctx context.Context, ddl *model.DDLEvent } return nil }, retry.WithPreExecutionWhenRetry(func() error { - return m.connector.SwitchToAvailableMySQLDB(ctx) + return m.connector.SwitchToAnAvailableDB(ctx) }), retry.WithBackoffBaseDelay(pmysql.BackoffBaseDelay.Milliseconds()), retry.WithBackoffMaxDelay(pmysql.BackoffMaxDelay.Milliseconds()), retry.WithMaxTries(defaultDDLMaxRetry), diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index e8fe00969da..a231fc572c7 100644 --- a/cdc/sink/dmlsink/txn/mysql/mysql.go +++ b/cdc/sink/dmlsink/txn/mysql/mysql.go @@ -797,7 +797,7 @@ func (s *mysqlBackend) execDMLWithMaxRetries(pctx context.Context, dmls *prepare zap.Int("numOfRows", dmls.rowCount)) return nil }, retry.WithPreExecutionWhenRetry(func() error { - return s.connector.SwitchToAvailableMySQLDB(pctx) + return s.connector.SwitchToAnAvailableDB(pctx) }), retry.WithBackoffBaseDelay(pmysql.BackoffBaseDelay.Milliseconds()), retry.WithBackoffMaxDelay(pmysql.BackoffMaxDelay.Milliseconds()), retry.WithMaxTries(s.dmlMaxRetry), diff --git a/cdc/syncpointstore/mysql_syncpoint_store.go b/cdc/syncpointstore/mysql_syncpoint_store.go index cd46828a0a1..1dfba69102b 100644 --- a/cdc/syncpointstore/mysql_syncpoint_store.go +++ b/cdc/syncpointstore/mysql_syncpoint_store.go @@ -149,7 +149,7 @@ func (s *mysqlSyncPointStore) CreateSyncTable(ctx context.Context) error { err = tx.Commit() return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;")) }, retry.WithPreExecutionWhenRetry(func() error { - return s.connector.SwitchToAvailableMySQLDB(ctx) + return s.connector.SwitchToAnAvailableDB(ctx) }), retry.WithMaxTries(2)) } @@ -226,7 +226,7 @@ func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context, err = tx.Commit() return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;")) }, retry.WithPreExecutionWhenRetry(func() error { - return s.connector.SwitchToAvailableMySQLDB(ctx) + return s.connector.SwitchToAnAvailableDB(ctx) }), retry.WithMaxTries(2)) } diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go index a536927b6c8..f8860f57e20 100644 --- a/pkg/sink/mysql/mysql_connector.go +++ b/pkg/sink/mysql/mysql_connector.go @@ -23,7 +23,7 @@ import ( ) // DBConnector manages the database connection, handling reconnections -// in case of connection failures when calling SwitchToAvailableMySQLDB method. +// in case of connection failures when calling SwitchToAnAvailableDB method. // To execute SQL queries or interact with the database, use the CurrentDB field of // DBConnector to access the underlying *sql.DB instance, i.e., DBConnector.CurrentDB. type DBConnector struct { @@ -44,7 +44,7 @@ type DBConnector struct { // configureDBWhenSwitch stores the function that will be automatically invoked // to configure a new MySQL connection after switching to it using the - // SwitchToAvailableMySQLDB function. This function will be set by the + // SwitchToAnAvailableDB function. This function will be set by the // ConfigureDBWhenSwitch method and can be applied to both newly established // connections and the current active connection if required. configureDBWhenSwitch func() @@ -77,7 +77,7 @@ func NewDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *url.UR connector := &DBConnector{dsnList: dsnList, nextTry: 0, dbConnFactory: dbConnFactory} - err = connector.SwitchToAvailableMySQLDB(ctx) + err = connector.SwitchToAnAvailableDB(ctx) if err != nil { return nil, err } @@ -85,10 +85,10 @@ func NewDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *url.UR return connector, nil } -// SwitchToAvailableMySQLDB attempts to switch to an available MySQL-compatible database connection +// SwitchToAnAvailableDB attempts to switch to an available MySQL-compatible database connection // if the current connection is invalid, and it updates DBConnector.CurrentDB for user to use it. // If there is only one DSN in DBConnector, switching is not possible. -func (c *DBConnector) SwitchToAvailableMySQLDB(ctx context.Context) error { +func (c *DBConnector) SwitchToAnAvailableDB(ctx context.Context) error { // If a connection has already been established and there is only one DSN (Data Source Name) available, // it is not possible to connect a different DSN. Therefore, simply return from the function. if len(c.dsnList) == 1 && c.CurrentDB != nil { @@ -131,7 +131,7 @@ func (c *DBConnector) SwitchToAvailableMySQLDB(ctx context.Context) error { } // ConfigureDBWhenSwitch allows for automatic configuration of a new MySQL connection -// when switching to an available connection using the SwitchToAvailableMySQLDB function. +// when switching to an available connection using the SwitchToAnAvailableDB function. // By providing the function `f` as an argument, it ensures that any newly established // connection is automatically configured after the switch. Additionally, if the existing // DBConnector.CurrentDB also requires configuration, you can set diff --git a/pkg/sink/mysql/mysql_connector_test.go b/pkg/sink/mysql/mysql_connector_test.go index a3c8041570e..acdb50a2f55 100644 --- a/pkg/sink/mysql/mysql_connector_test.go +++ b/pkg/sink/mysql/mysql_connector_test.go @@ -106,8 +106,8 @@ func TestNewDBConnectorWithFactory_generateDSNsFail(t *testing.T) { require.Equal(t, numCallStandard, 0) } -// Test SwitchToAvailableMySQLDB when current DB is valid -func TestSwitchToAvailableMySQLDB_CurrentDBValid(t *testing.T) { +// Test SwitchToAnAvailableDB when current DB is valid +func TestSwitchToAnAvailableDB_CurrentDBValid(t *testing.T) { ctx := context.Background() sinkURI, _ := url.Parse("mysql://user:password@localhost,localhost,localhost") cfg := NewConfig() @@ -129,14 +129,14 @@ func TestSwitchToAvailableMySQLDB_CurrentDBValid(t *testing.T) { require.NotNil(t, connector.CurrentDB) dbBeforeSwitch := connector.CurrentDB - require.NoError(t, connector.SwitchToAvailableMySQLDB(ctx)) + require.NoError(t, connector.SwitchToAnAvailableDB(ctx)) dbAfterSwitch := connector.CurrentDB require.Equal(t, dbBeforeSwitch, dbAfterSwitch) require.NoError(t, connector.CurrentDB.Close()) } -// Test SwitchToAvailableMySQLDB when current DB is invalid and switches to a new DB -func TestSwitchToAvailableMySQLDB_SwitchDB(t *testing.T) { +// Test SwitchToAnAvailableDB when current DB is invalid and switches to a new DB +func TestSwitchToAnAvailableDB_SwitchDB(t *testing.T) { ctx := context.Background() sinkURI, _ := url.Parse("mysql://user:password@localhost:123,localhost:456,localhost:789") cfg := NewConfig() @@ -175,7 +175,7 @@ func TestSwitchToAvailableMySQLDB_SwitchDB(t *testing.T) { require.NoError(t, dbCandidates[i].Close()) require.Error(t, dbCandidates[i].Ping()) if i != len(dbCandidates)-1 { - require.NoError(t, connector.SwitchToAvailableMySQLDB(ctx)) + require.NoError(t, connector.SwitchToAnAvailableDB(ctx)) require.NoError(t, connector.CurrentDB.Ping()) dbAfterSwitch := connector.CurrentDB require.NotEqual(t, dbBeforeSwitch, dbAfterSwitch) @@ -214,7 +214,7 @@ func TestConfigureDBWhenSwitch_Success(t *testing.T) { require.NoError(t, connector.CurrentDB.Ping()) require.NoError(t, connector.CurrentDB.Close()) - require.NoError(t, connector.SwitchToAvailableMySQLDB(ctx)) + require.NoError(t, connector.SwitchToAnAvailableDB(ctx)) require.NoError(t, connector.CurrentDB.Ping()) require.NoError(t, connector.CurrentDB.Close()) require.Equal(t, numCallConfigure, 2) @@ -244,6 +244,6 @@ func TestConfigureDBWhenSwitch_NilConfigureFunction(t *testing.T) { connector.ConfigureDBWhenSwitch(nil, true) require.NoError(t, connector.CurrentDB.Close()) - require.NoError(t, connector.SwitchToAvailableMySQLDB(ctx)) + require.NoError(t, connector.SwitchToAnAvailableDB(ctx)) require.NoError(t, connector.CurrentDB.Close()) } diff --git a/pkg/sink/observer/tidb.go b/pkg/sink/observer/tidb.go index ee5eb00e796..707b9b035ea 100644 --- a/pkg/sink/observer/tidb.go +++ b/pkg/sink/observer/tidb.go @@ -258,6 +258,6 @@ func queryMetrics[T metricColumnImpl, F metricColumnIface[T]]( } return nil }, retry.WithPreExecutionWhenRetry(func() error { - return connector.SwitchToAvailableMySQLDB(ctx) + return connector.SwitchToAnAvailableDB(ctx) }), retry.WithMaxTries(2)) } From 78a80a41f944e5fb07b0f462fbfb0911f11a99be Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 28 Aug 2024 15:05:54 +0800 Subject: [PATCH 38/50] test(multi_down_addresses): add `cdc cli changefeed update` integration test --- .../conf/diff_config_1.toml | 2 +- .../conf/diff_config_2.toml | 2 +- .../conf/diff_config_3.toml | 2 +- .../multi_down_addresses/data/prepare.sql | 9 +- .../multi_down_addresses/run.sh | 93 ++++++++++++++++--- 5 files changed, 89 insertions(+), 19 deletions(-) diff --git a/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml b/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml index c94b682a3dd..cf69c4232e2 100644 --- a/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml +++ b/tests/integration_tests/multi_down_addresses/conf/diff_config_1.toml @@ -13,7 +13,7 @@ check-struct-only = false target-instance = "down" - target-check-tables = ["multi_down_addresses.round"] + target-check-tables = ["multi_down_addresses.?*"] [data-sources] [data-sources.up] diff --git a/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml b/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml index 27ed498ee7c..d56a2d8cf48 100644 --- a/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml +++ b/tests/integration_tests/multi_down_addresses/conf/diff_config_2.toml @@ -13,7 +13,7 @@ check-struct-only = false target-instance = "down" - target-check-tables = ["multi_down_addresses.round"] + target-check-tables = ["multi_down_addresses.?*"] [data-sources] [data-sources.up] diff --git a/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml b/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml index 542b91ac42d..2709c3aa06a 100644 --- a/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml +++ b/tests/integration_tests/multi_down_addresses/conf/diff_config_3.toml @@ -13,7 +13,7 @@ check-struct-only = false target-instance = "down" - target-check-tables = ["multi_down_addresses.round"] + target-check-tables = ["multi_down_addresses.?*"] [data-sources] [data-sources.up] diff --git a/tests/integration_tests/multi_down_addresses/data/prepare.sql b/tests/integration_tests/multi_down_addresses/data/prepare.sql index 845c4571833..110d7c484f5 100644 --- a/tests/integration_tests/multi_down_addresses/data/prepare.sql +++ b/tests/integration_tests/multi_down_addresses/data/prepare.sql @@ -1,5 +1,6 @@ -drop database if exists multi_down_addresses; -create database multi_down_addresses; -use multi_down_addresses; +DROP DATABASE IF EXISTS multi_down_addresses; +CREATE DATABASE multi_down_addresses; +USE multi_down_addresses; -CREATE TABLE round (round INT PRIMARY KEY, val INT); +CREATE TABLE create (round INT PRIMARY KEY, val INT); +CREATE TABLE update (round INT PRIMARY KEY, val INT); diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh index d0d037aa859..2574f08dc07 100644 --- a/tests/integration_tests/multi_down_addresses/run.sh +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -13,7 +13,9 @@ function prepare() { rm -rf $WORK_DIR && mkdir -p $WORK_DIR start_tidb_cluster --workdir $WORK_DIR --downstream_db 0 cd $WORK_DIR +} +function prepare_create() { # Start downstream TiDB instances start_downstream_tidb_instances --db 3 --out_dir $WORK_DIR mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" @@ -21,15 +23,17 @@ function prepare() { # Start the CDC synchronization task. run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - run_cdc_cli changefeed create --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_2},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_3}/" + run_cdc_cli changefeed create \ + --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_2},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_3}/" \ + --changefeed-id="multi_down_addresses" sleep 5 - # Prepare a table + # Prepare tables for test `cdc cli changefeed create` and `cdc cli changefeed update` run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists multi_down_addresses.round ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 + check_table_exists multi_down_addresses.create ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 } -function run() { +function run_create() { pid1=${down_tidb_pids[0]} pid2=${down_tidb_pids[1]} pid3=${down_tidb_pids[2]} @@ -39,7 +43,7 @@ function run() { # tidb 2 should works kill $pid1 run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.round VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.create VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml @@ -49,7 +53,7 @@ function run() { run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 kill $pid2 - run_sql "INSERT INTO multi_down_addresses.round VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.create VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml @@ -57,7 +61,7 @@ function run() { # begin -> insert -> recover tidb 1 -> shutdown tidb 3 -> commit -> check_sync_diff # tidb 1 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.round VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.create VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" @@ -70,13 +74,13 @@ function run() { # begin -> insert -> recover tidb 2 -> shutdown tidb 1 -> insert -> commit -> check_sync_diff # tidb 2 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.round VALUES(4, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.create VALUES(4, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid2=${down_tidb_pids[0]} kill $pid1 - run_sql "INSERT INTO multi_down_addresses.round VALUES(41, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.create VALUES(41, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml @@ -84,21 +88,86 @@ function run() { # begin -> insert -> commit -> recover tidb 3 -> shutdown tidb 2 -> begin -> insert -> commit -> check_sync_diff # tidb 3 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.round VALUES(5, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.create VALUES(5, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 3 + pid3=${down_tidb_pids[0]} kill $pid2 - run_sql "INSERT INTO multi_down_addresses.round VALUES(51, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.create VALUES(51, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml + + kill $pid3 +} + +function prepare_update() { + # Start downstream TiDB instances + start_downstream_tidb_instances --db 2 --out_dir $WORK_DIR + mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" + echo "Started downstream TiDB instances with PIDs: ${down_tidb_pids[@]}" + + run_cdc_cli changefeed pause -c "multi_down_addresses" + run_cdc_cli changefeed update \ + -c "multi_down_addresses" \ + --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_2}/" + run_cdc_cli changefeed resume -c "multi_down_addresses" + sleep 5 + + check_table_exists multi_down_addresses.update ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 +} + +function run_update() { + pid1=${down_tidb_pids[0]} + pid2=${down_tidb_pids[1]} + + # Round 1 + # shutdown tidb 1 -> begin -> insert -> commit -> check_sync_diff + # tidb 2 should works + kill $pid1 + run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.update VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml + + # Round 2 + # begin -> insert -> recover tidb 1 -> shutdown tidb 2 -> commit -> check_sync_diff + # tidb 1 should works + run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.update VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + sleep 5 + start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 + mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" + pid1=${down_tidb_pids[0]} + kill $pid2 + run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/conf/diff_config_1.toml + + # Round 3 + # begin -> insert -> recover tidb 2 -> shutdown tidb 1 -> insert -> commit -> check_sync_diff + # tidb 2 should works + run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.update VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + sleep 5 + start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 + mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" + pid2=${down_tidb_pids[0]} + kill $pid1 + run_sql "INSERT INTO multi_down_addresses.create VALUES(31, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml } # No need to support kafka and storage sink. if [ "$SINK_TYPE" == "mysql" ]; then trap stop_tidb_cluster EXIT + prepare $* - run $* + prepare_create $* + run_create $* + prepare_update $* + run_update $* + check_logs $WORK_DIR echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" fi From aec17b3e862dfd4dbd882cf6d1d7b3bf01ddde10 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 28 Aug 2024 15:18:25 +0800 Subject: [PATCH 39/50] fix(multi_down_addresses): modify the `changefeed-id` according to the naming convention for changefeed-id --- tests/integration_tests/multi_down_addresses/run.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh index 2574f08dc07..1b72fd442a8 100644 --- a/tests/integration_tests/multi_down_addresses/run.sh +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -25,7 +25,7 @@ function prepare_create() { run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY run_cdc_cli changefeed create \ --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_2},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_3}/" \ - --changefeed-id="multi_down_addresses" + --changefeed-id="multi-down-addresses" sleep 5 # Prepare tables for test `cdc cli changefeed create` and `cdc cli changefeed update` @@ -107,11 +107,11 @@ function prepare_update() { mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" echo "Started downstream TiDB instances with PIDs: ${down_tidb_pids[@]}" - run_cdc_cli changefeed pause -c "multi_down_addresses" + run_cdc_cli changefeed pause -c "multi-down-addresses" run_cdc_cli changefeed update \ - -c "multi_down_addresses" \ + -c "multi-down-addresses" \ --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_2}/" - run_cdc_cli changefeed resume -c "multi_down_addresses" + run_cdc_cli changefeed resume -c "multi-down-addresses" sleep 5 check_table_exists multi_down_addresses.update ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 From 97db8ddaddd18b07ee1a8d68a31b40c052ac0c30 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 28 Aug 2024 15:25:40 +0800 Subject: [PATCH 40/50] fix(multi_down_addresses): fix a naming error of tables --- .../multi_down_addresses/data/prepare.sql | 4 +-- .../multi_down_addresses/run.sh | 26 +++++++++---------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/integration_tests/multi_down_addresses/data/prepare.sql b/tests/integration_tests/multi_down_addresses/data/prepare.sql index 110d7c484f5..75666d304fa 100644 --- a/tests/integration_tests/multi_down_addresses/data/prepare.sql +++ b/tests/integration_tests/multi_down_addresses/data/prepare.sql @@ -2,5 +2,5 @@ DROP DATABASE IF EXISTS multi_down_addresses; CREATE DATABASE multi_down_addresses; USE multi_down_addresses; -CREATE TABLE create (round INT PRIMARY KEY, val INT); -CREATE TABLE update (round INT PRIMARY KEY, val INT); +CREATE TABLE changefeed_create (round INT PRIMARY KEY, val INT); +CREATE TABLE changefeed_update (round INT PRIMARY KEY, val INT); diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh index 1b72fd442a8..dc06d63f382 100644 --- a/tests/integration_tests/multi_down_addresses/run.sh +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -30,7 +30,7 @@ function prepare_create() { # Prepare tables for test `cdc cli changefeed create` and `cdc cli changefeed update` run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists multi_down_addresses.create ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 + check_table_exists multi_down_addresses.changefeed_create ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 } function run_create() { @@ -43,7 +43,7 @@ function run_create() { # tidb 2 should works kill $pid1 run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.create VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml @@ -53,7 +53,7 @@ function run_create() { run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 kill $pid2 - run_sql "INSERT INTO multi_down_addresses.create VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml @@ -61,7 +61,7 @@ function run_create() { # begin -> insert -> recover tidb 1 -> shutdown tidb 3 -> commit -> check_sync_diff # tidb 1 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.create VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" @@ -74,13 +74,13 @@ function run_create() { # begin -> insert -> recover tidb 2 -> shutdown tidb 1 -> insert -> commit -> check_sync_diff # tidb 2 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.create VALUES(4, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(4, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid2=${down_tidb_pids[0]} kill $pid1 - run_sql "INSERT INTO multi_down_addresses.create VALUES(41, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(41, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml @@ -88,13 +88,13 @@ function run_create() { # begin -> insert -> commit -> recover tidb 3 -> shutdown tidb 2 -> begin -> insert -> commit -> check_sync_diff # tidb 3 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.create VALUES(5, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(5, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 3 pid3=${down_tidb_pids[0]} kill $pid2 - run_sql "INSERT INTO multi_down_addresses.create VALUES(51, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(51, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml @@ -114,7 +114,7 @@ function prepare_update() { run_cdc_cli changefeed resume -c "multi-down-addresses" sleep 5 - check_table_exists multi_down_addresses.update ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 + check_table_exists multi_down_addresses.changefeed_update ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 } function run_update() { @@ -126,7 +126,7 @@ function run_update() { # tidb 2 should works kill $pid1 run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.update VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_update VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml @@ -134,7 +134,7 @@ function run_update() { # begin -> insert -> recover tidb 1 -> shutdown tidb 2 -> commit -> check_sync_diff # tidb 1 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.update VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_update VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" @@ -147,13 +147,13 @@ function run_update() { # begin -> insert -> recover tidb 2 -> shutdown tidb 1 -> insert -> commit -> check_sync_diff # tidb 2 should works run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.update VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_update VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid2=${down_tidb_pids[0]} kill $pid1 - run_sql "INSERT INTO multi_down_addresses.create VALUES(31, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(31, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml } From 3673ef8856badbb6ff4b303f999fc1040d939fbf Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 28 Aug 2024 16:26:12 +0800 Subject: [PATCH 41/50] test(multi_down_addresses): add DDL SQL into `multi_down_addresses` integration test for testing DDL Sink when handling connection error --- .../multi_down_addresses/run.sh | 65 +++++++++---------- 1 file changed, 32 insertions(+), 33 deletions(-) diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh index dc06d63f382..7000f8795bc 100644 --- a/tests/integration_tests/multi_down_addresses/run.sh +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -31,71 +31,71 @@ function prepare_create() { # Prepare tables for test `cdc cli changefeed create` and `cdc cli changefeed update` run_sql_file $CUR/data/prepare.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_table_exists multi_down_addresses.changefeed_create ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 + check_table_exists multi_down_addresses.changefeed_update ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 } +# Test after `cdc cli changefeed create` function run_create() { pid1=${down_tidb_pids[0]} pid2=${down_tidb_pids[1]} pid3=${down_tidb_pids[2]} # Round 1 - # shutdown tidb 1 -> begin -> insert -> commit -> check_sync_diff + # shutdown tidb 1 -> insert -> insert -> check_sync_diff # tidb 2 should works kill $pid1 - run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(1, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml # Round 2 - # begin -> shutdown tidb 2 -> insert -> commit -> check_sync_diff + # insert -> shutdown tidb 2 -> update -> update -> check_sync_diff # tidb 3 should works - run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 kill $pid2 - run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "UPDATE multi_down_addresses.changefeed_create SET val=2 WHERE round=2;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "UPDATE multi_down_addresses.changefeed_create SET val=3 WHERE round=2;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml # Round 3 - # begin -> insert -> recover tidb 1 -> shutdown tidb 3 -> commit -> check_sync_diff + # insert -> update -> recover tidb 1 -> shutdown tidb 3 -> update -> check_sync_diff # tidb 1 should works - run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "UPDATE multi_down_addresses.changefeed_create SET val=2 WHERE round=3;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid1=${down_tidb_pids[0]} kill $pid3 - run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "UPDATE multi_down_addresses.changefeed_create SET val=3 WHERE round=3;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_1.toml # Round 4 - # begin -> insert -> recover tidb 2 -> shutdown tidb 1 -> insert -> commit -> check_sync_diff + # insert -> insert -> recover tidb 2 -> shutdown tidb 1 -> update -> delete -> check_sync_diff # tidb 2 should works - run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(4, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(41, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid2=${down_tidb_pids[0]} kill $pid1 - run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(41, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "UPDATE multi_down_addresses.changefeed_create SET val=2 WHERE round=4;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "DELETE FROM multi_down_addresses.changefeed_create WHERE round=41;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml # Round 5 - # begin -> insert -> commit -> recover tidb 3 -> shutdown tidb 2 -> begin -> insert -> commit -> check_sync_diff + # insert -> update -> recover tidb 3 -> shutdown tidb 2 -> insert -> delete -> check_sync_diff # tidb 3 should works - run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(5, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "UPDATE multi_down_addresses.changefeed_create SET val=2 WHERE round=5;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 3 pid3=${down_tidb_pids[0]} kill $pid2 - run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(51, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(51, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "DELETE FROM multi_down_addresses.changefeed_create WHERE round=51;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml kill $pid3 @@ -113,48 +113,47 @@ function prepare_update() { --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_2}/" run_cdc_cli changefeed resume -c "multi-down-addresses" sleep 5 - - check_table_exists multi_down_addresses.changefeed_update ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT_1} 60 } +# Test after `cdc cli changefeed update` function run_update() { pid1=${down_tidb_pids[0]} pid2=${down_tidb_pids[1]} # Round 1 - # shutdown tidb 1 -> begin -> insert -> commit -> check_sync_diff + # shutdown tidb 1 -> insert -> insert -> insert -> check_sync_diff # tidb 2 should works kill $pid1 - run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO multi_down_addresses.changefeed_update VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_update VALUES(11, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_update VALUES(111, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml # Round 2 - # begin -> insert -> recover tidb 1 -> shutdown tidb 2 -> commit -> check_sync_diff + # create -> insert -> recover tidb 1 -> shutdown tidb 2 -> drop -> check_sync_diff # tidb 1 should works - run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.changefeed_update VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE TABLE multi_down_addresses.ddl1 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.ddl1 VALUES(2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid1=${down_tidb_pids[0]} kill $pid2 - run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "DROP TABLE multi_down_addresses.ddl1;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_1.toml # Round 3 - # begin -> insert -> recover tidb 2 -> shutdown tidb 1 -> insert -> commit -> check_sync_diff + # create -> create -> recover tidb 2 -> shutdown tidb 1 -> create -> drop -> check_sync_diff # tidb 2 should works - run_sql "BEGIN;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.changefeed_update VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE TABLE multi_down_addresses.ddl3 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE TABLE multi_down_addresses.ddl33 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid2=${down_tidb_pids[0]} kill $pid1 - run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(31, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "COMMIT;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE TABLE multi_down_addresses.ddl333 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "DROP TABLE multi_down_addresses.ddl3;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml } From e1c6a437992eb12534e0a5c150c164a935cddfa3 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Wed, 28 Aug 2024 16:32:48 +0800 Subject: [PATCH 42/50] fix(multi_down_addresses): fix a wrong SQL by duplicate entry for one key --- tests/integration_tests/multi_down_addresses/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh index 7000f8795bc..8ffdacfa51d 100644 --- a/tests/integration_tests/multi_down_addresses/run.sh +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -45,7 +45,7 @@ function run_create() { # tidb 2 should works kill $pid1 run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(1, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(11, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml # Round 2 From 13edec0a9ffc73436eadcc3e65c5f03374bd8e76 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Thu, 29 Aug 2024 17:06:03 +0800 Subject: [PATCH 43/50] fix(DDL Sink): fix `IsRetryableDDLError` function to allow DDL to be retryable when encounter network error --- pkg/errorutil/util.go | 2 +- pkg/sink/mysql/db_helper.go | 14 ++++++++++++-- pkg/sink/mysql/mysql_connector.go | 3 ++- 3 files changed, 15 insertions(+), 4 deletions(-) diff --git a/pkg/errorutil/util.go b/pkg/errorutil/util.go index 5834d5c24cb..4297e5edff4 100644 --- a/pkg/errorutil/util.go +++ b/pkg/errorutil/util.go @@ -124,7 +124,7 @@ func IsRetryableDDLError(err error) bool { err = errors.Cause(err) mysqlErr, ok := err.(*gmysql.MySQLError) if !ok { - return false + return true } // If the error is in the black list, return false. diff --git a/pkg/sink/mysql/db_helper.go b/pkg/sink/mysql/db_helper.go index 4ad04df1855..bffad004bbf 100644 --- a/pkg/sink/mysql/db_helper.go +++ b/pkg/sink/mysql/db_helper.go @@ -40,6 +40,8 @@ import ( // is retrieved, the temporary connection is closed. The retrieved data is then // used to populate additional parameters into the Sink URI, refining // the connection URL. +// NOTE: the function returns no error if not all of the DSNs are unavailable and +// will report warnings if some of the DSNs are unavailable. func generateDSNs(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory ConnectionFactory) ([]string, error) { dsnCfgs, err := GetDSNCfgs(sinkURI, cfg) if err != nil { @@ -47,17 +49,23 @@ func generateDSNs(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFact } var oneOfErrs error + oneAvailable := false dsnList := make([]string, len(dsnCfgs)) for i := 0; i < len(dsnCfgs); i++ { - // dsn format of the driver: + // DSN format of the driver: // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] dsnList[i], err = checkAndGenerateDSNByConfig(ctx, dsnCfgs[i], cfg, dbConnFactory) if err != nil { oneOfErrs = err + } else { + oneAvailable = true } } - return dsnList, oneOfErrs + if !oneAvailable { + return nil, oneOfErrs + } + return dsnList, nil } func checkAndGenerateDSNByConfig(ctx context.Context, dsnCfg *dmysql.Config, cfg *Config, dbConnFactory ConnectionFactory) (string, error) { @@ -69,8 +77,10 @@ func checkAndGenerateDSNByConfig(ctx context.Context, dsnCfg *dmysql.Config, cfg dsnStr, err := generateDSNByConfig(ctx, dsnCfg, cfg, testDB) if err != nil { + log.Warn("DSN is invaild", zap.String("DSN", dsnCfg.FormatDSN())) return "", err } + log.Debug("DSN is vaild", zap.String("DSN", dsnStr)) return dsnStr, nil } diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go index f8860f57e20..7f37228fa8d 100644 --- a/pkg/sink/mysql/mysql_connector.go +++ b/pkg/sink/mysql/mysql_connector.go @@ -65,11 +65,12 @@ func NewDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *url.UR dbConnFactory = &DBConnectionFactory{} } + log.Debug("create connection by sinkURI", zap.String("sinkURI", sinkURI.String())) + // generateDSNs function parses multiple addresses from the URL (if any) // and generates a DSN (Data Source Name) for each one. // For each DSN, the function attempts to create a connection and perform a Ping // to verify its availability. - // If any of the DSNs are unavailable, the function immediately returns an error. dsnList, err := generateDSNs(ctx, sinkURI, cfg, dbConnFactory.CreateTemporaryConnection) if err != nil { return nil, err From bac745d487aa831562c36c2408f763dd20406bbe Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Thu, 29 Aug 2024 19:02:11 +0800 Subject: [PATCH 44/50] feat(connector): add log output --- pkg/sink/mysql/mysql_connector.go | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go index 7f37228fa8d..21319671428 100644 --- a/pkg/sink/mysql/mysql_connector.go +++ b/pkg/sink/mysql/mysql_connector.go @@ -16,6 +16,7 @@ package mysql import ( "context" "database/sql" + "fmt" "net/url" "github.com/pingcap/log" @@ -93,6 +94,8 @@ func (c *DBConnector) SwitchToAnAvailableDB(ctx context.Context) error { // If a connection has already been established and there is only one DSN (Data Source Name) available, // it is not possible to connect a different DSN. Therefore, simply return from the function. if len(c.dsnList) == 1 && c.CurrentDB != nil { + log.Debug("cannot switch to other db server, only one address was given", + zap.String("DSN", c.dsnList[c.curIndex()])) return nil } @@ -100,6 +103,8 @@ func (c *DBConnector) SwitchToAnAvailableDB(ctx context.Context) error { // Check if the current connection is available; return immediately if it is. err := c.CurrentDB.PingContext(ctx) if err == nil { + log.Debug("current connection is vaild, no need to switch", + zap.String("DSN", c.dsnList[c.curIndex()])) return nil } @@ -119,15 +124,23 @@ func (c *DBConnector) SwitchToAnAvailableDB(ctx context.Context) error { db, err := c.dbConnFactory.CreateStandardConnection(ctx, c.dsnList[c.nextTry]) c.nextTry = (c.nextTry + 1) % len(c.dsnList) + if err == nil { c.CurrentDB = db if c.configureDBWhenSwitch != nil { c.configureDBWhenSwitch() } + log.Info( + fmt.Sprintf("current connection is invaild, switch to the %d-th address", c.curIndex()), + zap.String("DSN", c.dsnList[c.curIndex()])) return nil + } else { + log.Debug(fmt.Sprintf("try to switch the %d-th addresses, but it's invaild", c.curIndex()), + zap.String("DSN", c.dsnList[c.curIndex()])) } } + log.Error("fail to switch an available db server, all of the given addresses are invaild") return err } @@ -147,3 +160,11 @@ func (c *DBConnector) ConfigureDBWhenSwitch(f func(), needConfigureCurrentDB boo c.configureDBWhenSwitch() } } + +func (c *DBConnector) curIndex() int { + cur := c.nextTry - 1 + if cur < 0 { + cur = len(c.dsnList) - 1 + } + return cur +} From fc445c0821422324107a7ca1f6b62252f294ab66 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Thu, 29 Aug 2024 19:27:30 +0800 Subject: [PATCH 45/50] refactor(connector): modify the log output of `DBConnctor` --- pkg/sink/mysql/mysql_connector.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go index 21319671428..66250ead92a 100644 --- a/pkg/sink/mysql/mysql_connector.go +++ b/pkg/sink/mysql/mysql_connector.go @@ -131,11 +131,11 @@ func (c *DBConnector) SwitchToAnAvailableDB(ctx context.Context) error { c.configureDBWhenSwitch() } log.Info( - fmt.Sprintf("current connection is invaild, switch to the %d-th address", c.curIndex()), + fmt.Sprintf("switch to the %d-th address", c.curIndex()+1), zap.String("DSN", c.dsnList[c.curIndex()])) return nil } else { - log.Debug(fmt.Sprintf("try to switch the %d-th addresses, but it's invaild", c.curIndex()), + log.Debug(fmt.Sprintf("try to switch the %d-th addresses, but it's invaild", c.curIndex()+1), zap.String("DSN", c.dsnList[c.curIndex()])) } } From 9aeeeca73b3cf695042794c5c90314b3375d9850 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Thu, 29 Aug 2024 19:31:32 +0800 Subject: [PATCH 46/50] refactor(multi_down_addresses): modify `multi_down_addresses` integration test --- .../multi_down_addresses/run.sh | 40 +++++++++++-------- 1 file changed, 24 insertions(+), 16 deletions(-) diff --git a/tests/integration_tests/multi_down_addresses/run.sh b/tests/integration_tests/multi_down_addresses/run.sh index 8ffdacfa51d..7b4e422e184 100644 --- a/tests/integration_tests/multi_down_addresses/run.sh +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -42,7 +42,7 @@ function run_create() { # Round 1 # shutdown tidb 1 -> insert -> insert -> check_sync_diff - # tidb 2 should works + # tidb 2 should work kill $pid1 run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(11, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} @@ -50,9 +50,8 @@ function run_create() { # Round 2 # insert -> shutdown tidb 2 -> update -> update -> check_sync_diff - # tidb 3 should works + # tidb 3 should work run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - sleep 5 kill $pid2 run_sql "UPDATE multi_down_addresses.changefeed_create SET val=2 WHERE round=2;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "UPDATE multi_down_addresses.changefeed_create SET val=3 WHERE round=2;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} @@ -60,10 +59,9 @@ function run_create() { # Round 3 # insert -> update -> recover tidb 1 -> shutdown tidb 3 -> update -> check_sync_diff - # tidb 1 should works + # tidb 1 should work run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(3, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "UPDATE multi_down_addresses.changefeed_create SET val=2 WHERE round=3;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid1=${down_tidb_pids[0]} @@ -73,10 +71,9 @@ function run_create() { # Round 4 # insert -> insert -> recover tidb 2 -> shutdown tidb 1 -> update -> delete -> check_sync_diff - # tidb 2 should works + # tidb 2 should work run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(4, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(41, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid2=${down_tidb_pids[0]} @@ -87,10 +84,9 @@ function run_create() { # Round 5 # insert -> update -> recover tidb 3 -> shutdown tidb 2 -> insert -> delete -> check_sync_diff - # tidb 3 should works + # tidb 3 should work run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(5, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "UPDATE multi_down_addresses.changefeed_create SET val=2 WHERE round=5;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 3 pid3=${down_tidb_pids[0]} kill $pid2 @@ -98,7 +94,9 @@ function run_create() { run_sql "DELETE FROM multi_down_addresses.changefeed_create WHERE round=51;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml + # Kill all tidb instances kill $pid3 + sleep 5 } function prepare_update() { @@ -110,7 +108,8 @@ function prepare_update() { run_cdc_cli changefeed pause -c "multi-down-addresses" run_cdc_cli changefeed update \ -c "multi-down-addresses" \ - --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_2}/" + --sink-uri="mysql://root@${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1},${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_2}/" \ + --no-confirm run_cdc_cli changefeed resume -c "multi-down-addresses" sleep 5 } @@ -122,7 +121,7 @@ function run_update() { # Round 1 # shutdown tidb 1 -> insert -> insert -> insert -> check_sync_diff - # tidb 2 should works + # tidb 2 should work kill $pid1 run_sql "INSERT INTO multi_down_addresses.changefeed_update VALUES(1, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO multi_down_addresses.changefeed_update VALUES(11, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} @@ -131,10 +130,9 @@ function run_update() { # Round 2 # create -> insert -> recover tidb 1 -> shutdown tidb 2 -> drop -> check_sync_diff - # tidb 1 should works + # tidb 1 should work run_sql "CREATE TABLE multi_down_addresses.ddl1 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO multi_down_addresses.ddl1 VALUES(2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid1=${down_tidb_pids[0]} @@ -144,10 +142,9 @@ function run_update() { # Round 3 # create -> create -> recover tidb 2 -> shutdown tidb 1 -> create -> drop -> check_sync_diff - # tidb 2 should works + # tidb 2 should work run_sql "CREATE TABLE multi_down_addresses.ddl3 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE TABLE multi_down_addresses.ddl33 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - sleep 5 start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 2 mapfile -t down_tidb_pids <"$WORK_DIR/downstream_tidb_instances_pids.log" pid2=${down_tidb_pids[0]} @@ -155,6 +152,17 @@ function run_update() { run_sql "CREATE TABLE multi_down_addresses.ddl333 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "DROP TABLE multi_down_addresses.ddl3;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml + + # Round 4 + # create -> create -> recover tidb 1 -> shutdown tidb 2 -> create -> drop -> check_sync_diff + # tidb 1 should work + run_sql "CREATE TABLE multi_down_addresses.ddl4 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE TABLE multi_down_addresses.ddl44 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + start_downstream_tidb_instances --db 1 --out_dir $WORK_DIR --suffix 1 + kill $pid2 + run_sql "CREATE TABLE multi_down_addresses.ddl444 (round INT PRIMARY KEY)" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "DROP TABLE multi_down_addresses.ddl44;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_sync_diff $WORK_DIR $CUR/conf/diff_config_1.toml } # No need to support kafka and storage sink. @@ -166,7 +174,7 @@ if [ "$SINK_TYPE" == "mysql" ]; then run_create $* prepare_update $* run_update $* - + check_logs $WORK_DIR echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" fi From 757d8d041a3af262bfd9f0ca7877904eef7bc018 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 30 Aug 2024 11:27:29 +0800 Subject: [PATCH 47/50] refactor(connector): modify the log output of `DBConnctor` --- pkg/sink/mysql/mysql_connector.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/pkg/sink/mysql/mysql_connector.go b/pkg/sink/mysql/mysql_connector.go index 66250ead92a..42dbe6c66b5 100644 --- a/pkg/sink/mysql/mysql_connector.go +++ b/pkg/sink/mysql/mysql_connector.go @@ -103,7 +103,7 @@ func (c *DBConnector) SwitchToAnAvailableDB(ctx context.Context) error { // Check if the current connection is available; return immediately if it is. err := c.CurrentDB.PingContext(ctx) if err == nil { - log.Debug("current connection is vaild, no need to switch", + log.Debug("current connection is vaild, no need to switch a downstream db server", zap.String("DSN", c.dsnList[c.curIndex()])) return nil } @@ -131,13 +131,15 @@ func (c *DBConnector) SwitchToAnAvailableDB(ctx context.Context) error { c.configureDBWhenSwitch() } log.Info( - fmt.Sprintf("switch to the %d-th address", c.curIndex()+1), + fmt.Sprintf("switch to the %d-th address of downstream db server", c.curIndex()+1), zap.String("DSN", c.dsnList[c.curIndex()])) return nil - } else { - log.Debug(fmt.Sprintf("try to switch the %d-th addresses, but it's invaild", c.curIndex()+1), - zap.String("DSN", c.dsnList[c.curIndex()])) } + log.Warn( + fmt.Sprintf( + "try to switch the %d-th addresses of downstream db server, but it's invaild", + c.curIndex()+1), + zap.String("DSN", c.dsnList[c.curIndex()])) } log.Error("fail to switch an available db server, all of the given addresses are invaild") From d5f4fa727855454fd8c75e461e2ad496873df5ec Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 30 Aug 2024 14:25:06 +0800 Subject: [PATCH 48/50] fix(DDL Sink): fix the unit test failure of `TestIsRetryableDDLError` --- pkg/errorutil/util_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/errorutil/util_test.go b/pkg/errorutil/util_test.go index 9c0d1d65034..0b8a74ad6f5 100644 --- a/pkg/errorutil/util_test.go +++ b/pkg/errorutil/util_test.go @@ -98,7 +98,7 @@ func TestIsRetryableDDLError(t *testing.T) { err error ret bool }{ - {errors.New("raw error"), false}, + {errors.New("raw error"), true}, {newMysqlErr(tmysql.ErrNoDB, "Error: Duplicate key name 'some_key'"), false}, {newMysqlErr(tmysql.ErrParse, "Can't create database"), false}, {newMysqlErr(tmysql.ErrAccessDenied, "Access denied for user"), false}, From 3dbfef1c441f7e9ec771d797dcb35833327a6f38 Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 30 Aug 2024 15:43:49 +0800 Subject: [PATCH 49/50] refactor(test_prepare): modify `TLS_TIDB_PORT` from 3307 to 3406 --- tests/integration_tests/_utils/test_prepare | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/_utils/test_prepare b/tests/integration_tests/_utils/test_prepare index c59889da0ad..3133200b69e 100644 --- a/tests/integration_tests/_utils/test_prepare +++ b/tests/integration_tests/_utils/test_prepare @@ -11,7 +11,7 @@ DOWN_TIDB_STATUS_2=${DOWN_TIDB_STATUS_2:-20081} DOWN_TIDB_PORT_3=${DOWN_TIDB_PORT_3:-3308} DOWN_TIDB_STATUS_3=${DOWN_TIDB_STATUS_3:-20082} TLS_TIDB_HOST=${TLS_TIDB_HOST:-127.0.0.1} -TLS_TIDB_PORT=${TLS_TIDB_PORT:-3310} +TLS_TIDB_PORT=${TLS_TIDB_PORT:-3406} TLS_TIDB_STATUS=${TLS_TIDB_STATUS:-30080} UP_PD_HOST_1=${UP_PD_HOST_1:-127.0.0.1} From d80463347ffa21a724f8dd39e89893cbc0dd9d0b Mon Sep 17 00:00:00 2001 From: wlwilliamx Date: Fri, 30 Aug 2024 17:33:29 +0800 Subject: [PATCH 50/50] refactor(DDL Sink): remove leftover TODO tag --- cdc/owner/ddl_sink.go | 1 - 1 file changed, 1 deletion(-) diff --git a/cdc/owner/ddl_sink.go b/cdc/owner/ddl_sink.go index 083e89ab124..ecbe8c3444a 100644 --- a/cdc/owner/ddl_sink.go +++ b/cdc/owner/ddl_sink.go @@ -391,7 +391,6 @@ func (s *ddlSinkImpl) emitSyncPoint(ctx context.Context, checkpointTs uint64) (e for { if err = s.makeSyncPointStoreReady(ctx); err == nil { // TODO implement async sink syncPoint - // TODO: err = s.syncPointStore.SinkSyncPoint(ctx, s.changefeedID, checkpointTs) } if err == nil {