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/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go index dfac1654577..dd7a7df106a 100644 --- a/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go +++ b/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go @@ -15,12 +15,12 @@ package mysql import ( "context" - "database/sql" "fmt" "net/url" "time" lru "github.com/hashicorp/golang-lru" + perrors "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" timodel "github.com/pingcap/tidb/pkg/parser/model" @@ -54,10 +54,9 @@ 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 - // db is the database connection. - db *sql.DB - cfg *pmysql.Config + id model.ChangeFeedID + connector *pmysql.DBConnector + cfg *pmysql.Config // statistics is the statistics of this sink. // We use it to record the DDL count. statistics *metrics.Statistics @@ -81,22 +80,17 @@ func NewDDLSink( return nil, err } - dsnStr, err := pmysql.GenerateDSN(ctx, sinkURI, cfg, GetDBConnImpl.CreateTemporaryConnection) + connector, err := pmysql.NewDBConnectorWithFactory(ctx, cfg, sinkURI, GetDBConnImpl) if err != nil { return nil, err } - db, err := GetDBConnImpl.CreateStandardConnection(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 } @@ -107,7 +101,7 @@ func NewDDLSink( } m := &DDLSink{ id: changefeedID, - db: db, + connector: connector, cfg: cfg, statistics: metrics.NewStatistics(ctx, changefeedID, sink.TxnSink), lastExecutedNormalDDLCache: lruCache, @@ -157,7 +151,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.SwitchToAnAvailableDB(ctx) + }), retry.WithBackoffBaseDelay(pmysql.BackoffBaseDelay.Milliseconds()), retry.WithBackoffMaxDelay(pmysql.BackoffMaxDelay.Milliseconds()), retry.WithMaxTries(defaultDDLMaxRetry), retry.WithIsRetryableErr(errorutil.IsRetryableDDLError)) @@ -210,7 +206,7 @@ func (m *DDLSink) execDDL(pctx context.Context, ddl *model.DDLEvent) error { start := time.Now() log.Info("Start exec DDL", zap.String("namespace", m.id.Namespace), zap.String("changefeed", m.id.ID), zap.Uint64("commitTs", ddl.CommitTs), zap.String("DDL", ddl.Query)) - tx, err := m.db.BeginTx(ctx, nil) + tx, err := m.connector.CurrentDB.BeginTx(ctx, nil) if err != nil { return err } @@ -229,7 +225,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)) @@ -282,8 +278,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), diff --git a/cdc/sink/dmlsink/txn/mysql/mysql.go b/cdc/sink/dmlsink/txn/mysql/mysql.go index ac11a198014..a231fc572c7 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 - db *sql.DB + connector pmysql.DBConnector cfg *pmysql.Config dmlMaxRetry uint64 @@ -97,47 +97,44 @@ func NewMySQLBackends( return nil, err } - dsnStr, err := pmysql.GenerateDSN(ctx, sinkURI, cfg, dbConnFactory.CreateTemporaryConnection) + connector, err := pmysql.NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) if err != nil { return nil, err } - db, err := dbConnFactory.CreateStandardConnection(ctx, dsnStr) + cfg.IsTiDB, err = pmysql.CheckIsTiDB(ctx, connector.CurrentDB) if err != nil { return nil, err } - cfg.IsTiDB, err = pmysql.CheckIsTiDB(ctx, db) + cfg.IsWriteSourceExisted, err = pmysql.CheckIfBDRModeIsSupported(ctx, connector.CurrentDB) if err != nil { return nil, err } - cfg.IsWriteSourceExisted, err = pmysql.CheckIfBDRModeIsSupported(ctx, db) - if err != nil { - 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 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 } @@ -167,7 +164,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), @@ -180,7 +177,7 @@ func NewMySQLBackends( backends = append(backends, &mysqlBackend{ workerID: i, changefeed: changefeed, - db: db, + connector: *connector, cfg: cfg, dmlMaxRetry: defaultDMLMaxRetry, statistics: statistics, @@ -261,9 +258,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.CurrentDB = nil } return } @@ -673,7 +670,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 { @@ -741,7 +738,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), @@ -799,7 +796,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.SwitchToAnAvailableDB(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 161002e3381..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.GenBasicDSN(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/cdc/syncpointstore/mysql_syncpoint_store.go b/cdc/syncpointstore/mysql_syncpoint_store.go index 5b56601cb9c..1dfba69102b 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" @@ -27,6 +26,7 @@ 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" pmysql "github.com/pingcap/tiflow/pkg/sink/mysql" "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" @@ -34,8 +34,8 @@ import ( type mysqlSyncPointStore struct { id model.ChangeFeedID - db *sql.DB cfg *pmysql.Config + connector *pmysql.DBConnector clusterID string syncPointRetention time.Duration lastCleanSyncPointTime time.Time @@ -54,22 +54,16 @@ func newMySQLSyncPointStore( return nil, err } - dsnStr, err := pmysql.GenerateDSN(ctx, sinkURI, cfg, pmysql.CreateMySQLDBConn) + connector, err := pmysql.NewDBConnector(ctx, cfg, sinkURI) if err != nil { return nil, err } - syncDB, err := pmysql.CreateMySQLDBConn(ctx, dsnStr) + cfg.IsTiDB, err = pmysql.CheckIsTiDB(ctx, connector.CurrentDB) if err != nil { return nil, err } - - cfg.IsTiDB, err = pmysql.CheckIsTiDB(ctx, syncDB) - if err != nil { - return nil, err - } - - cfg.IsWriteSourceExisted, err = pmysql.CheckIfBDRModeIsSupported(ctx, syncDB) + cfg.IsWriteSourceExisted, err = pmysql.CheckIfBDRModeIsSupported(ctx, connector.CurrentDB) if err != nil { return nil, err } @@ -80,8 +74,8 @@ func newMySQLSyncPointStore( return &mysqlSyncPointStore{ id: changefeedID, - db: syncDB, cfg: cfg, + connector: connector, clusterID: config.GetGlobalServerConfig().ClusterID, syncPointRetention: util.GetOrZero(replicaConfig.SyncPointRetention), lastCleanSyncPointTime: time.Now(), @@ -89,48 +83,49 @@ 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.String("namespace", s.id.Namespace), - zap.String("changefeed", s.id.ID), zap.Error(err)) - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "create sync table: begin Tx fail;")) - } + 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.String("namespace", s.id.Namespace), + zap.String("changefeed", s.id.ID), zap.Error(err)) + return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "create sync table: begin Tx fail;")) + } - // we try to set cdc write source for the ddl - if err = pmysql.SetWriteSource(ctx, s.cfg, tx); err != nil { - if rbErr := tx.Rollback(); rbErr != nil { - if errors.Cause(rbErr) != context.Canceled { - log.Error("Failed to rollback", - zap.String("namespace", s.id.Namespace), - zap.String("changefeed", s.id.ID), zap.Error(err)) + // we try to set cdc write source for the ddl + if err = pmysql.SetWriteSource(ctx, s.cfg, tx); err != nil { + if rbErr := tx.Rollback(); rbErr != nil { + if errors.Cause(rbErr) != context.Canceled { + log.Error("Failed to rollback", + zap.String("namespace", s.id.Namespace), + zap.String("changefeed", s.id.ID), zap.Error(err)) + } } + return err } - return err - } - _, 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.String("namespace", s.id.Namespace), - zap.String("changefeed", s.id.ID), 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.String("namespace", s.id.Namespace), + zap.String("changefeed", s.id.ID), 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;")) - } - _, err = tx.Exec("USE " + database) - if err != nil { - err2 := tx.Rollback() - if err2 != nil { - log.Error("failed to create syncpoint table", - zap.String("namespace", s.id.Namespace), - zap.String("changefeed", s.id.ID), 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.String("namespace", s.id.Namespace), + zap.String("changefeed", s.id.ID), 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 + query := `CREATE TABLE IF NOT EXISTS %s ( ticdc_cluster_id varchar (255), changefeed varchar(255), @@ -140,95 +135,102 @@ func (s *mysqlSyncPointStore) CreateSyncTable(ctx context.Context) error { 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.String("namespace", s.id.Namespace), - zap.String("changefeed", s.id.ID), zap.Error(err2)) + 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.String("namespace", s.id.Namespace), + zap.String("changefeed", s.id.ID), 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.SwitchToAnAvailableDB(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;")) + } } - } - err = tx.Commit() - 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;")) + }, retry.WithPreExecutionWhenRetry(func() error { + return s.connector.SwitchToAnAvailableDB(ctx) + }), retry.WithMaxTries(2)) } func (s *mysqlSyncPointStore) Close() error { - err := s.db.Close() + err := s.connector.CurrentDB.Close() return cerror.WrapError(cerror.ErrMySQLConnectionError, err) } 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/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}, diff --git a/pkg/retry/options.go b/pkg/retry/options.go index 942f040253f..5e9d7dcfb90 100644 --- a/pkg/retry/options.go +++ b/pkg/retry/options.go @@ -33,12 +33,21 @@ 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 { 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 +103,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 } 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 eb1dd8288ca..bffad004bbf 100644 --- a/pkg/sink/mysql/db_helper.go +++ b/pkg/sink/mysql/db_helper.go @@ -18,71 +18,71 @@ 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" ) -// GenerateDSN generates the dsn with the given config. -// GenerateDSN 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 GenerateDSN(ctx context.Context, sinkURI *url.URL, cfg *Config, dbConnFactory ConnectionFactory) (dsnStr string, err error) { - // dsn format of the driver: - // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] - dsn, err := GenBasicDSN(sinkURI, cfg) +// 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 { - return "", err + return nil, err } - var testDB *sql.DB - testDB, err = GetTestDB(ctx, dsn, dbConnFactory) - if err != nil { - return + var oneOfErrs error + oneAvailable := false + dsnList := make([]string, len(dsnCfgs)) + for i := 0; i < len(dsnCfgs); i++ { + // 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 + } } - 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 + if !oneAvailable { + return nil, oneOfErrs } - // NOTE: quote the string is necessary to avoid ambiguities. - dsn.Params["sql_mode"] = strconv.Quote(dsn.Params["sql_mode"]) + return dsnList, nil +} - dsnStr, err = generateDSNByConfig(ctx, dsn, cfg, testDB) +func checkAndGenerateDSNByConfig(ctx context.Context, dsnCfg *dmysql.Config, cfg *Config, dbConnFactory ConnectionFactory) (string, error) { + testDB, err := GetTestDB(ctx, dsnCfg, dbConnFactory) if err != nil { - return + return "", err } + defer testDB.Close() - // check if GBK charset is supported by downstream - var gbkSupported bool - gbkSupported, err = checkCharsetSupport(ctx, testDB, charset.CharsetGBK) + dsnStr, err := generateDSNByConfig(ctx, dsnCfg, cfg, testDB) 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)) + log.Warn("DSN is invaild", zap.String("DSN", dsnCfg.FormatDSN())) + return "", err } + log.Debug("DSN is vaild", zap.String("DSN", dsnStr)) - return + return dsnStr, nil } func generateDSNByConfig( @@ -107,6 +107,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 @@ -156,6 +166,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())) @@ -222,8 +244,12 @@ func GetTestDB(ctx context.Context, dbConfig *dmysql.Config, dbConnFactory Conne return testDB, err } -// GenBasicDSN generates a basic DSN from the given config. -func GenBasicDSN(sinkURI *url.URL, cfg *Config) (*dmysql.Config, error) { +// 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] username := sinkURI.User.Username() @@ -232,33 +258,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) - } + 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 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 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 dsnCfgs[i].Params == nil { + dsnCfgs[i].Params = make(map[string]string, 1) + } + if cfg.Timezone != "" { + dsnCfgs[i].Params["time_zone"] = cfg.Timezone + } + dsnCfgs[i].Params["readTimeout"] = cfg.ReadTimeout + dsnCfgs[i].Params["writeTimeout"] = cfg.WriteTimeout + dsnCfgs[i].Params["timeout"] = cfg.DialTimeout } - if cfg.Timezone != "" { - dsn.Params["time_zone"] = cfg.Timezone + return dsnCfgs, 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 { + // 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" + } + 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. @@ -330,7 +369,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/pkg/sink/mysql/mock_db.go b/pkg/sink/mysql/mock_db.go index 67ba74c1901..571e94bf9a1 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 new file mode 100644 index 00000000000..42dbe6c66b5 --- /dev/null +++ b/pkg/sink/mysql/mysql_connector.go @@ -0,0 +1,172 @@ +// 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 ( + "context" + "database/sql" + "fmt" + "net/url" + + "github.com/pingcap/log" + "go.uber.org/zap" +) + +// DBConnector manages the database connection, handling reconnections +// 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 { + // 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(). + // 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 + + // dbConnFactory is a Factory function for DBConnector that creates database connections. + dbConnFactory IDBConnectionFactory + + // configureDBWhenSwitch stores the function that will be automatically invoked + // to configure a new MySQL connection after switching to it using 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() +} + +// 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{}) +} + +// 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 NewDBConnectorWithFactory(ctx context.Context, cfg *Config, sinkURI *url.URL, dbConnFactory IDBConnectionFactory) (*DBConnector, error) { + if dbConnFactory == nil { + 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. + dsnList, err := generateDSNs(ctx, sinkURI, cfg, dbConnFactory.CreateTemporaryConnection) + if err != nil { + return nil, err + } + + connector := &DBConnector{dsnList: dsnList, nextTry: 0, dbConnFactory: dbConnFactory} + + err = connector.SwitchToAnAvailableDB(ctx) + if err != nil { + return nil, err + } + + return connector, nil +} + +// 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) 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 + } + + if c.CurrentDB != nil { + // 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 a downstream db server", + zap.String("DSN", c.dsnList[c.curIndex()])) + 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 := 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("switch to the %d-th address of downstream db server", c.curIndex()+1), + zap.String("DSN", c.dsnList[c.curIndex()])) + return nil + } + 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") + return err +} + +// ConfigureDBWhenSwitch allows for automatic configuration of a new MySQL connection +// 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 +// `needConfigureCurrentDB` to true, and this function will automatically apply +// the configuration function `f` to it as well. +func (c *DBConnector) ConfigureDBWhenSwitch(f func(), needConfigureCurrentDB bool) { + if f == nil { + return + } + c.configureDBWhenSwitch = f + if needConfigureCurrentDB { + c.configureDBWhenSwitch() + } +} + +func (c *DBConnector) curIndex() int { + cur := c.nextTry - 1 + if cur < 0 { + cur = len(c.dsnList) - 1 + } + return cur +} diff --git a/pkg/sink/mysql/mysql_connector_test.go b/pkg/sink/mysql/mysql_connector_test.go new file mode 100644 index 00000000000..acdb50a2f55 --- /dev/null +++ b/pkg/sink/mysql/mysql_connector_test.go @@ -0,0 +1,249 @@ +// 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 ( + "context" + "database/sql" + "errors" + "net/url" + "testing" + + "github.com/DATA-DOG/go-sqlmock" + "github.com/stretchr/testify/require" +) + +// 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() + + 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 := NewDBConnectorWithFactory(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 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") + 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 := NewDBConnectorWithFactory(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 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() + + 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 := NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + require.Error(t, err) + require.Nil(t, connector) + require.Equal(t, numCallStandard, 0) +} + +// 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() + + 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 := NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + require.NoError(t, err) + require.NotNil(t, connector) + require.NotNil(t, connector.CurrentDB) + dbBeforeSwitch := connector.CurrentDB + + require.NoError(t, connector.SwitchToAnAvailableDB(ctx)) + dbAfterSwitch := connector.CurrentDB + require.Equal(t, dbBeforeSwitch, dbAfterSwitch) + require.NoError(t, connector.CurrentDB.Close()) +} + +// 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() + + 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, err + } + + 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 := NewDBConnectorWithFactory(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.SwitchToAnAvailableDB(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_Success(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 := NewDBConnectorWithFactory(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.SwitchToAnAvailableDB(ctx)) + require.NoError(t, connector.CurrentDB.Ping()) + 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 := NewDBConnectorWithFactory(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.SwitchToAnAvailableDB(ctx)) + require.NoError(t, connector.CurrentDB.Close()) +} diff --git a/pkg/sink/observer/observer.go b/pkg/sink/observer/observer.go index 348ca6573f6..356ef1070a0 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" ) @@ -38,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 } @@ -60,14 +59,14 @@ 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) } 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) @@ -81,25 +80,23 @@ func NewObserver( return nil, err } - dsnStr, err := pmysql.GenerateDSN(ctx, sinkURI, cfg, options.dbConnFactory) + connector, err := pmysql.NewDBConnectorWithFactory(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/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.go b/pkg/sink/observer/tidb.go index 8a39faca836..707b9b035ea 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.DBConnector } // 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.DBConnector) *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.DBConnector, 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.SwitchToAnAvailableDB(ctx) + }), retry.WithMaxTries(2)) } diff --git a/pkg/sink/observer/tidb_test.go b/pkg/sink/observer/tidb_test.go index 7303209e457..0b78a04cab7 100644 --- a/pkg/sink/observer/tidb_test.go +++ b/pkg/sink/observer/tidb_test.go @@ -15,63 +15,110 @@ 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" ) func TestTiDBObserver(t *testing.T) { t.Parallel() - db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) - require.NoError(t, err) + 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(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(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(queryConnCountStmt). - WillReturnRows( - sqlmock.NewRows( - []string{"time", "instance", "value"}). - AddRow("2023-01-10 16:44:39.123000", "10.2.6.127:11080", 24), - ) + 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(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(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(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(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() - observer := NewTiDBObserver(db) + connector, err := mysql.NewDBConnectorWithFactory(ctx, cfg, sinkURI, dbConnFactory) + 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() +// } diff --git a/tests/integration_tests/_utils/start_downstream_tidb_instances b/tests/integration_tests/_utils/start_downstream_tidb_instances new file mode 100755 index 00000000000..c393961cd60 --- /dev/null +++ b/tests/integration_tests/_utils/start_downstream_tidb_instances @@ -0,0 +1,110 @@ +#!/bin/bash + +# 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= + +while [[ ${1} ]]; do + case "${1}" in + --db) + db=${2} + shift + ;; + --out_dir) + out_dir=${2} + shift + ;; + --suffix) + suffix=${2} + shift + ;; + *) + echo "Unknown parameter: ${1}" >&2 + exit 1 + ;; + esac + + if ! shift; then + echo 'Missing parameter argument.' >&2 + exit 1 + fi +done + +if [ $((db + suffix - 1)) -gt 3 ]; then + echo 'The integration test only supports a maximum of 3 downstream TiDB instances' + exit 1 +fi + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +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() { + 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" +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 +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 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 diff --git a/tests/integration_tests/_utils/start_tidb_cluster b/tests/integration_tests/_utils/start_tidb_cluster index 3113f7a2cbd..ed1d874f244 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 @@ -60,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 diff --git a/tests/integration_tests/_utils/start_tidb_cluster_impl b/tests/integration_tests/_utils/start_tidb_cluster_impl index 7c306179836..67000df4602 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 @@ -243,17 +248,6 @@ tidb-server \ --status=${UP_TIDB_OTHER_STATUS} \ --log-file "$OUT_DIR/tidb_other.log" & -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" & - 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 @@ -275,22 +269,9 @@ while ! mysql -uroot -h${UP_TIDB_HOST} -P${UP_TIDB_OTHER_PORT} --default-charact sleep 2 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 -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} -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} 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_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 diff --git a/tests/integration_tests/_utils/test_prepare b/tests/integration_tests/_utils/test_prepare index b08a0664864..3133200b69e 100644 --- a/tests/integration_tests/_utils/test_prepare +++ b/tests/integration_tests/_utils/test_prepare @@ -4,10 +4,14 @@ 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} +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:-3406} 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..6f585ad0aca 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,8 +33,8 @@ 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 \ - -a "${DOWN_TIDB_HOST}:${DOWN_TIDB_STATUS}" + -d "root@tcp(${DOWN_TIDB_HOST}:${DOWN_TIDB_PORT_1})/bank" --test-round=20000 \ + -a "${DOWN_TIDB_HOST}:${DOWN_TIDB_STATUS_1}" cleanup_process $CDC_BINARY echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" 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 6cbba476da9..e1003c1c42a 100644 --- a/tests/integration_tests/bdr_mode/run.sh +++ b/tests/integration_tests/bdr_mode/run.sh @@ -35,16 +35,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_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..cf69c4232e2 --- /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 = ["multi_down_addresses.?*"] + +[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..d56a2d8cf48 --- /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 = ["multi_down_addresses.?*"] + +[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..2709c3aa06a --- /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 = ["multi_down_addresses.?*"] + +[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..75666d304fa --- /dev/null +++ b/tests/integration_tests/multi_down_addresses/data/prepare.sql @@ -0,0 +1,6 @@ +DROP DATABASE IF EXISTS multi_down_addresses; +CREATE DATABASE multi_down_addresses; +USE multi_down_addresses; + +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 new file mode 100644 index 00000000000..7b4e422e184 --- /dev/null +++ b/tests/integration_tests/multi_down_addresses/run.sh @@ -0,0 +1,180 @@ +#!/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 +} + +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" + 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}/" \ + --changefeed-id="multi-down-addresses" + sleep 5 + + # 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 -> insert -> insert -> check_sync_diff + # 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} + check_sync_diff $WORK_DIR $CUR/conf/diff_config_2.toml + + # Round 2 + # insert -> shutdown tidb 2 -> update -> update -> check_sync_diff + # tidb 3 should work + run_sql "INSERT INTO multi_down_addresses.changefeed_create VALUES(2, 1);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + 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} + check_sync_diff $WORK_DIR $CUR/conf/diff_config_3.toml + + # Round 3 + # insert -> update -> recover tidb 1 -> shutdown tidb 3 -> update -> check_sync_diff + # 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} + 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 "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 + # insert -> insert -> recover tidb 2 -> shutdown tidb 1 -> update -> delete -> check_sync_diff + # 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} + 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 "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 + # insert -> update -> recover tidb 3 -> shutdown tidb 2 -> insert -> delete -> check_sync_diff + # 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} + 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, 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 all tidb instances + kill $pid3 + sleep 5 +} + +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}/" \ + --no-confirm + run_cdc_cli changefeed resume -c "multi-down-addresses" + sleep 5 +} + +# Test after `cdc cli changefeed update` +function run_update() { + pid1=${down_tidb_pids[0]} + pid2=${down_tidb_pids[1]} + + # Round 1 + # shutdown tidb 1 -> insert -> insert -> insert -> check_sync_diff + # 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} + 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 + # create -> insert -> recover tidb 1 -> shutdown tidb 2 -> drop -> check_sync_diff + # 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} + 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 "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 + # create -> create -> recover tidb 2 -> shutdown tidb 1 -> create -> drop -> check_sync_diff + # 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} + 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 "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. +if [ "$SINK_TYPE" == "mysql" ]; then + trap stop_tidb_cluster EXIT + + prepare $* + prepare_create $* + run_create $* + prepare_update $* + run_update $* + + check_logs $WORK_DIR + echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" +fi 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/run_group.sh b/tests/integration_tests/run_group.sh index 991ba3d188a..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 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