github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/mysql.go (about)

     1  // Copyright 2020 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package sink
    15  
    16  import (
    17  	"context"
    18  	"database/sql"
    19  	"fmt"
    20  	"net/url"
    21  	"runtime"
    22  	"strconv"
    23  	"strings"
    24  	"sync"
    25  	"sync/atomic"
    26  	"time"
    27  
    28  	dmysql "github.com/go-sql-driver/mysql"
    29  	"github.com/pingcap/errors"
    30  	"github.com/pingcap/failpoint"
    31  	"github.com/pingcap/log"
    32  	timodel "github.com/pingcap/parser/model"
    33  	"github.com/pingcap/parser/mysql"
    34  	"github.com/pingcap/ticdc/cdc/model"
    35  	"github.com/pingcap/ticdc/cdc/sink/common"
    36  	"github.com/pingcap/ticdc/pkg/config"
    37  	"github.com/pingcap/ticdc/pkg/cyclic"
    38  	"github.com/pingcap/ticdc/pkg/cyclic/mark"
    39  	cerror "github.com/pingcap/ticdc/pkg/errors"
    40  	"github.com/pingcap/ticdc/pkg/filter"
    41  	tifilter "github.com/pingcap/ticdc/pkg/filter"
    42  	"github.com/pingcap/ticdc/pkg/notify"
    43  	"github.com/pingcap/ticdc/pkg/quotes"
    44  	"github.com/pingcap/ticdc/pkg/retry"
    45  	"github.com/pingcap/ticdc/pkg/security"
    46  	"github.com/pingcap/ticdc/pkg/util"
    47  	tddl "github.com/pingcap/tidb/ddl"
    48  	"github.com/pingcap/tidb/infoschema"
    49  	"github.com/prometheus/client_golang/prometheus"
    50  	"go.uber.org/zap"
    51  )
    52  
    53  const (
    54  	defaultWorkerCount         = 16
    55  	defaultMaxTxnRow           = 256
    56  	defaultDMLMaxRetryTime     = 8
    57  	defaultDDLMaxRetryTime     = 20
    58  	defaultTiDBTxnMode         = "optimistic"
    59  	defaultFlushInterval       = time.Millisecond * 50
    60  	defaultBatchReplaceEnabled = true
    61  	defaultBatchReplaceSize    = 20
    62  	defaultReadTimeout         = "2m"
    63  	defaultWriteTimeout        = "2m"
    64  	defaultDialTimeout         = "2m"
    65  	defaultSafeMode            = true
    66  )
    67  
    68  const (
    69  	backoffBaseDelayInMs = 500
    70  	// in previous/backoff retry pkg, the DefaultMaxInterval = 60 * time.Second
    71  	backoffMaxDelayInMs = 60 * 1000
    72  )
    73  
    74  // SyncpointTableName is the name of table where all syncpoint maps sit
    75  const syncpointTableName string = "syncpoint_v1"
    76  
    77  var validSchemes = map[string]bool{
    78  	"mysql":     true,
    79  	"mysql+ssl": true,
    80  	"tidb":      true,
    81  	"tidb+ssl":  true,
    82  }
    83  
    84  type mysqlSyncpointStore struct {
    85  	db *sql.DB
    86  }
    87  
    88  type mysqlSink struct {
    89  	db     *sql.DB
    90  	params *sinkParams
    91  
    92  	filter *filter.Filter
    93  	cyclic *cyclic.Cyclic
    94  
    95  	txnCache      *common.UnresolvedTxnCache
    96  	workers       []*mysqlSinkWorker
    97  	resolvedTs    uint64
    98  	maxResolvedTs uint64
    99  
   100  	execWaitNotifier *notify.Notifier
   101  	resolvedNotifier *notify.Notifier
   102  	errCh            chan error
   103  	flushSyncWg      sync.WaitGroup
   104  
   105  	statistics *Statistics
   106  
   107  	// metrics used by mysql sink only
   108  	metricConflictDetectDurationHis prometheus.Observer
   109  	metricBucketSizeCounters        []prometheus.Counter
   110  
   111  	forceReplicate bool
   112  }
   113  
   114  func (s *mysqlSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error {
   115  	count := s.txnCache.Append(s.filter, rows...)
   116  	s.statistics.AddRowsCount(count)
   117  	return nil
   118  }
   119  
   120  // FlushRowChangedEvents will flushes all received events, we don't allow mysql
   121  // sink to receive events before resolving
   122  func (s *mysqlSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) {
   123  	if atomic.LoadUint64(&s.maxResolvedTs) < resolvedTs {
   124  		atomic.StoreUint64(&s.maxResolvedTs, resolvedTs)
   125  	}
   126  	// resolvedTs can be fallen back, such as a new table is added into this sink
   127  	// with a smaller start-ts
   128  	atomic.StoreUint64(&s.resolvedTs, resolvedTs)
   129  	s.resolvedNotifier.Notify()
   130  
   131  	// check and throw error
   132  	select {
   133  	case err := <-s.errCh:
   134  		return 0, err
   135  	default:
   136  	}
   137  
   138  	checkpointTs := resolvedTs
   139  	for _, worker := range s.workers {
   140  		workerCheckpointTs := atomic.LoadUint64(&worker.checkpointTs)
   141  		if workerCheckpointTs < checkpointTs {
   142  			checkpointTs = workerCheckpointTs
   143  		}
   144  	}
   145  	s.statistics.PrintStatus(ctx)
   146  	return checkpointTs, nil
   147  }
   148  
   149  func (s *mysqlSink) flushRowChangedEvents(ctx context.Context, receiver *notify.Receiver) {
   150  	defer func() {
   151  		for _, worker := range s.workers {
   152  			worker.closedCh <- struct{}{}
   153  		}
   154  	}()
   155  	for {
   156  		select {
   157  		case <-ctx.Done():
   158  			return
   159  		case <-receiver.C:
   160  		}
   161  		resolvedTs := atomic.LoadUint64(&s.resolvedTs)
   162  		resolvedTxnsMap := s.txnCache.Resolved(resolvedTs)
   163  		if len(resolvedTxnsMap) == 0 {
   164  			for _, worker := range s.workers {
   165  				atomic.StoreUint64(&worker.checkpointTs, resolvedTs)
   166  			}
   167  			s.txnCache.UpdateCheckpoint(resolvedTs)
   168  			continue
   169  		}
   170  
   171  		if !config.NewReplicaImpl && s.cyclic != nil {
   172  			// Filter rows if it is origined from downstream.
   173  			skippedRowCount := cyclic.FilterAndReduceTxns(
   174  				resolvedTxnsMap, s.cyclic.FilterReplicaID(), s.cyclic.ReplicaID())
   175  			s.statistics.SubRowsCount(skippedRowCount)
   176  		}
   177  		s.dispatchAndExecTxns(ctx, resolvedTxnsMap)
   178  		for _, worker := range s.workers {
   179  			atomic.StoreUint64(&worker.checkpointTs, resolvedTs)
   180  		}
   181  		s.txnCache.UpdateCheckpoint(resolvedTs)
   182  	}
   183  }
   184  
   185  func (s *mysqlSink) EmitCheckpointTs(ctx context.Context, ts uint64) error {
   186  	// do nothing
   187  	return nil
   188  }
   189  
   190  func (s *mysqlSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error {
   191  	if s.filter.ShouldIgnoreDDLEvent(ddl.StartTs, ddl.Type, ddl.TableInfo.Schema, ddl.TableInfo.Table) {
   192  		log.Info(
   193  			"DDL event ignored",
   194  			zap.String("query", ddl.Query),
   195  			zap.Uint64("startTs", ddl.StartTs),
   196  			zap.Uint64("commitTs", ddl.CommitTs),
   197  		)
   198  		return cerror.ErrDDLEventIgnored.GenWithStackByArgs()
   199  	}
   200  	s.statistics.AddDDLCount()
   201  	err := s.execDDLWithMaxRetries(ctx, ddl)
   202  	return errors.Trace(err)
   203  }
   204  
   205  // Initialize is no-op for Mysql sink
   206  func (s *mysqlSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error {
   207  	return nil
   208  }
   209  
   210  func (s *mysqlSink) execDDLWithMaxRetries(ctx context.Context, ddl *model.DDLEvent) error {
   211  	return retry.Do(ctx, func() error {
   212  		err := s.execDDL(ctx, ddl)
   213  		if isIgnorableDDLError(err) {
   214  			log.Info("execute DDL failed, but error can be ignored", zap.String("query", ddl.Query), zap.Error(err))
   215  			return nil
   216  		}
   217  		if err != nil {
   218  			log.Warn("execute DDL with error, retry later", zap.String("query", ddl.Query), zap.Error(err))
   219  		}
   220  		return err
   221  	}, retry.WithBackoffBaseDelay(backoffBaseDelayInMs), retry.WithBackoffMaxDelay(backoffMaxDelayInMs), retry.WithMaxTries(defaultDDLMaxRetryTime), retry.WithIsRetryableErr(cerror.IsRetryableError))
   222  }
   223  
   224  func (s *mysqlSink) execDDL(ctx context.Context, ddl *model.DDLEvent) error {
   225  	shouldSwitchDB := len(ddl.TableInfo.Schema) > 0 && ddl.Type != timodel.ActionCreateSchema
   226  
   227  	failpoint.Inject("MySQLSinkExecDDLDelay", func() {
   228  		select {
   229  		case <-ctx.Done():
   230  			failpoint.Return(ctx.Err())
   231  		case <-time.After(time.Hour):
   232  		}
   233  		failpoint.Return(nil)
   234  	})
   235  	err := s.statistics.RecordDDLExecution(func() error {
   236  		tx, err := s.db.BeginTx(ctx, nil)
   237  		if err != nil {
   238  			return err
   239  		}
   240  
   241  		if shouldSwitchDB {
   242  			_, err = tx.ExecContext(ctx, "USE "+quotes.QuoteName(ddl.TableInfo.Schema)+";")
   243  			if err != nil {
   244  				if rbErr := tx.Rollback(); rbErr != nil {
   245  					log.Error("Failed to rollback", zap.Error(err))
   246  				}
   247  				return err
   248  			}
   249  		}
   250  
   251  		if _, err = tx.ExecContext(ctx, ddl.Query); err != nil {
   252  			if rbErr := tx.Rollback(); rbErr != nil {
   253  				log.Error("Failed to rollback", zap.String("sql", ddl.Query), zap.Error(err))
   254  			}
   255  			return err
   256  		}
   257  
   258  		return tx.Commit()
   259  	})
   260  	if err != nil {
   261  		return cerror.WrapError(cerror.ErrMySQLTxnError, err)
   262  	}
   263  
   264  	log.Info("Exec DDL succeeded", zap.String("sql", ddl.Query))
   265  	return nil
   266  }
   267  
   268  // adjustSQLMode adjust sql mode according to sink config.
   269  func (s *mysqlSink) adjustSQLMode(ctx context.Context) error {
   270  	// Must relax sql mode to support cyclic replication, as downstream may have
   271  	// extra columns (not null and no default value).
   272  	if s.cyclic == nil || !s.cyclic.Enabled() {
   273  		return nil
   274  	}
   275  	var oldMode, newMode string
   276  	row := s.db.QueryRowContext(ctx, "SELECT @@SESSION.sql_mode;")
   277  	err := row.Scan(&oldMode)
   278  	if err != nil {
   279  		return cerror.WrapError(cerror.ErrMySQLQueryError, err)
   280  	}
   281  
   282  	newMode = cyclic.RelaxSQLMode(oldMode)
   283  	_, err = s.db.ExecContext(ctx, fmt.Sprintf("SET sql_mode = '%s';", newMode))
   284  	if err != nil {
   285  		return cerror.WrapError(cerror.ErrMySQLQueryError, err)
   286  	}
   287  	return nil
   288  }
   289  
   290  var _ Sink = &mysqlSink{}
   291  
   292  type sinkParams struct {
   293  	workerCount         int
   294  	maxTxnRow           int
   295  	tidbTxnMode         string
   296  	changefeedID        string
   297  	captureAddr         string
   298  	batchReplaceEnabled bool
   299  	batchReplaceSize    int
   300  	readTimeout         string
   301  	writeTimeout        string
   302  	dialTimeout         string
   303  	enableOldValue      bool
   304  	safeMode            bool
   305  	timezone            string
   306  	tls                 string
   307  }
   308  
   309  func (s *sinkParams) Clone() *sinkParams {
   310  	clone := *s
   311  	return &clone
   312  }
   313  
   314  var defaultParams = &sinkParams{
   315  	workerCount:         defaultWorkerCount,
   316  	maxTxnRow:           defaultMaxTxnRow,
   317  	tidbTxnMode:         defaultTiDBTxnMode,
   318  	batchReplaceEnabled: defaultBatchReplaceEnabled,
   319  	batchReplaceSize:    defaultBatchReplaceSize,
   320  	readTimeout:         defaultReadTimeout,
   321  	writeTimeout:        defaultWriteTimeout,
   322  	dialTimeout:         defaultDialTimeout,
   323  	safeMode:            defaultSafeMode,
   324  }
   325  
   326  func checkTiDBVariable(ctx context.Context, db *sql.DB, variableName, defaultValue string) (string, error) {
   327  	var name string
   328  	var value string
   329  	querySQL := fmt.Sprintf("show session variables like '%s';", variableName)
   330  	err := db.QueryRowContext(ctx, querySQL).Scan(&name, &value)
   331  	if err != nil && err != sql.ErrNoRows {
   332  		errMsg := "fail to query session variable " + variableName
   333  		return "", errors.Annotate(cerror.WrapError(cerror.ErrMySQLQueryError, err), errMsg)
   334  	}
   335  	// session variable works, use given default value
   336  	if err == nil {
   337  		return defaultValue, nil
   338  	}
   339  	// session variable not exists, return "" to ignore it
   340  	return "", nil
   341  }
   342  
   343  func configureSinkURI(
   344  	ctx context.Context,
   345  	dsnCfg *dmysql.Config,
   346  	params *sinkParams,
   347  	testDB *sql.DB,
   348  ) (string, error) {
   349  	if dsnCfg.Params == nil {
   350  		dsnCfg.Params = make(map[string]string, 1)
   351  	}
   352  	dsnCfg.DBName = ""
   353  	dsnCfg.InterpolateParams = true
   354  	dsnCfg.MultiStatements = true
   355  	// if timezone is empty string, we don't pass this variable in dsn
   356  	if params.timezone != "" {
   357  		dsnCfg.Params["time_zone"] = params.timezone
   358  	}
   359  	dsnCfg.Params["readTimeout"] = params.readTimeout
   360  	dsnCfg.Params["writeTimeout"] = params.writeTimeout
   361  	dsnCfg.Params["timeout"] = params.dialTimeout
   362  
   363  	autoRandom, err := checkTiDBVariable(ctx, testDB, "allow_auto_random_explicit_insert", "1")
   364  	if err != nil {
   365  		return "", err
   366  	}
   367  	if autoRandom != "" {
   368  		dsnCfg.Params["allow_auto_random_explicit_insert"] = autoRandom
   369  	}
   370  
   371  	txnMode, err := checkTiDBVariable(ctx, testDB, "tidb_txn_mode", params.tidbTxnMode)
   372  	if err != nil {
   373  		return "", err
   374  	}
   375  	if txnMode != "" {
   376  		dsnCfg.Params["tidb_txn_mode"] = txnMode
   377  	}
   378  
   379  	dsnClone := dsnCfg.Clone()
   380  	dsnClone.Passwd = "******"
   381  	log.Info("sink uri is configured", zap.String("format dsn", dsnClone.FormatDSN()))
   382  
   383  	return dsnCfg.FormatDSN(), nil
   384  }
   385  
   386  func parseSinkURI(ctx context.Context, sinkURI *url.URL, opts map[string]string) (*sinkParams, error) {
   387  	params := defaultParams.Clone()
   388  
   389  	if cid, ok := opts[OptChangefeedID]; ok {
   390  		params.changefeedID = cid
   391  	}
   392  	if caddr, ok := opts[OptCaptureAddr]; ok {
   393  		params.captureAddr = caddr
   394  	}
   395  
   396  	if sinkURI == nil {
   397  		return nil, cerror.ErrMySQLConnectionError.GenWithStack("fail to open MySQL sink, empty URL")
   398  	}
   399  	scheme := strings.ToLower(sinkURI.Scheme)
   400  	if _, ok := validSchemes[scheme]; !ok {
   401  		return nil, cerror.ErrMySQLConnectionError.GenWithStack("can't create mysql sink with unsupported scheme: %s", scheme)
   402  	}
   403  	s := sinkURI.Query().Get("worker-count")
   404  	if s != "" {
   405  		c, err := strconv.Atoi(s)
   406  		if err != nil {
   407  			return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err)
   408  		}
   409  		if c > 0 {
   410  			params.workerCount = c
   411  		}
   412  	}
   413  	s = sinkURI.Query().Get("max-txn-row")
   414  	if s != "" {
   415  		c, err := strconv.Atoi(s)
   416  		if err != nil {
   417  			return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err)
   418  		}
   419  		params.maxTxnRow = c
   420  	}
   421  	s = sinkURI.Query().Get("tidb-txn-mode")
   422  	if s != "" {
   423  		if s == "pessimistic" || s == "optimistic" {
   424  			params.tidbTxnMode = s
   425  		} else {
   426  			log.Warn("invalid tidb-txn-mode, should be pessimistic or optimistic, use optimistic as default")
   427  		}
   428  	}
   429  	if sinkURI.Query().Get("ssl-ca") != "" {
   430  		credential := security.Credential{
   431  			CAPath:   sinkURI.Query().Get("ssl-ca"),
   432  			CertPath: sinkURI.Query().Get("ssl-cert"),
   433  			KeyPath:  sinkURI.Query().Get("ssl-key"),
   434  		}
   435  		tlsCfg, err := credential.ToTLSConfig()
   436  		if err != nil {
   437  			return nil, errors.Annotate(err, "fail to open MySQL connection")
   438  		}
   439  		name := "cdc_mysql_tls" + params.changefeedID
   440  		err = dmysql.RegisterTLSConfig(name, tlsCfg)
   441  		if err != nil {
   442  			return nil, errors.Annotate(
   443  				cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection")
   444  		}
   445  		params.tls = "?tls=" + name
   446  	}
   447  
   448  	s = sinkURI.Query().Get("batch-replace-enable")
   449  	if s != "" {
   450  		enable, err := strconv.ParseBool(s)
   451  		if err != nil {
   452  			return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err)
   453  		}
   454  		params.batchReplaceEnabled = enable
   455  	}
   456  	if params.batchReplaceEnabled && sinkURI.Query().Get("batch-replace-size") != "" {
   457  		size, err := strconv.Atoi(sinkURI.Query().Get("batch-replace-size"))
   458  		if err != nil {
   459  			return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err)
   460  		}
   461  		params.batchReplaceSize = size
   462  	}
   463  
   464  	// TODO: force safe mode in startup phase
   465  	s = sinkURI.Query().Get("safe-mode")
   466  	if s != "" {
   467  		safeModeEnabled, err := strconv.ParseBool(s)
   468  		if err != nil {
   469  			return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err)
   470  		}
   471  		params.safeMode = safeModeEnabled
   472  	}
   473  
   474  	if _, ok := sinkURI.Query()["time-zone"]; ok {
   475  		s = sinkURI.Query().Get("time-zone")
   476  		if s == "" {
   477  			params.timezone = ""
   478  		} else {
   479  			params.timezone = fmt.Sprintf(`"%s"`, s)
   480  		}
   481  	} else {
   482  		tz := util.TimezoneFromCtx(ctx)
   483  		params.timezone = fmt.Sprintf(`"%s"`, tz.String())
   484  	}
   485  
   486  	// read, write, and dial timeout for each individual connection, equals to
   487  	// readTimeout, writeTimeout, timeout in go mysql driver respectively.
   488  	// ref: https://github.com/go-sql-driver/mysql#connection-pool-and-timeouts
   489  	// To keep the same style with other sink parameters, we use dash as word separator.
   490  	s = sinkURI.Query().Get("read-timeout")
   491  	if s != "" {
   492  		params.readTimeout = s
   493  	}
   494  	s = sinkURI.Query().Get("write-timeout")
   495  	if s != "" {
   496  		params.writeTimeout = s
   497  	}
   498  	s = sinkURI.Query().Get("timeout")
   499  	if s != "" {
   500  		params.dialTimeout = s
   501  	}
   502  
   503  	return params, nil
   504  }
   505  
   506  var getDBConnImpl = getDBConn
   507  
   508  func getDBConn(ctx context.Context, dsnStr string) (*sql.DB, error) {
   509  	db, err := sql.Open("mysql", dsnStr)
   510  	if err != nil {
   511  		return nil, errors.Annotate(
   512  			cerror.WrapError(cerror.ErrMySQLConnectionError, err), "Open database connection failed")
   513  	}
   514  	err = db.PingContext(ctx)
   515  	if err != nil {
   516  		// close db to recycle resources
   517  		if closeErr := db.Close(); closeErr != nil {
   518  			log.Warn("close db failed", zap.Error(err))
   519  		}
   520  		return nil, errors.Annotate(
   521  			cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection")
   522  	}
   523  	return db, nil
   524  }
   525  
   526  // newMySQLSink creates a new MySQL sink using schema storage
   527  func newMySQLSink(
   528  	ctx context.Context,
   529  	changefeedID model.ChangeFeedID,
   530  	sinkURI *url.URL,
   531  	filter *tifilter.Filter,
   532  	replicaConfig *config.ReplicaConfig,
   533  	opts map[string]string,
   534  ) (Sink, error) {
   535  	opts[OptChangefeedID] = changefeedID
   536  	params, err := parseSinkURI(ctx, sinkURI, opts)
   537  	if err != nil {
   538  		return nil, err
   539  	}
   540  
   541  	params.enableOldValue = replicaConfig.EnableOldValue
   542  
   543  	// dsn format of the driver:
   544  	// [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...&paramN=valueN]
   545  	username := sinkURI.User.Username()
   546  	password, _ := sinkURI.User.Password()
   547  	port := sinkURI.Port()
   548  	if username == "" {
   549  		username = "root"
   550  	}
   551  	if port == "" {
   552  		port = "4000"
   553  	}
   554  
   555  	dsnStr := fmt.Sprintf("%s:%s@tcp(%s:%s)/%s", username, password, sinkURI.Hostname(), port, params.tls)
   556  	dsn, err := dmysql.ParseDSN(dsnStr)
   557  	if err != nil {
   558  		return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err)
   559  	}
   560  
   561  	// create test db used for parameter detection
   562  	if dsn.Params == nil {
   563  		dsn.Params = make(map[string]string, 1)
   564  	}
   565  	if params.timezone != "" {
   566  		dsn.Params["time_zone"] = params.timezone
   567  	}
   568  	dsn.Params["readTimeout"] = params.readTimeout
   569  	dsn.Params["writeTimeout"] = params.writeTimeout
   570  	dsn.Params["timeout"] = params.dialTimeout
   571  	testDB, err := getDBConnImpl(ctx, dsn.FormatDSN())
   572  	if err != nil {
   573  		return nil, err
   574  	}
   575  	defer testDB.Close()
   576  
   577  	dsnStr, err = configureSinkURI(ctx, dsn, params, testDB)
   578  	if err != nil {
   579  		return nil, errors.Trace(err)
   580  	}
   581  	db, err := getDBConnImpl(ctx, dsnStr)
   582  	if err != nil {
   583  		return nil, err
   584  	}
   585  
   586  	log.Info("Start mysql sink")
   587  
   588  	db.SetMaxIdleConns(params.workerCount)
   589  	db.SetMaxOpenConns(params.workerCount)
   590  
   591  	metricConflictDetectDurationHis := conflictDetectDurationHis.WithLabelValues(
   592  		params.captureAddr, params.changefeedID)
   593  	metricBucketSizeCounters := make([]prometheus.Counter, params.workerCount)
   594  	for i := 0; i < params.workerCount; i++ {
   595  		metricBucketSizeCounters[i] = bucketSizeCounter.WithLabelValues(
   596  			params.captureAddr, params.changefeedID, strconv.Itoa(i))
   597  	}
   598  
   599  	sink := &mysqlSink{
   600  		db:                              db,
   601  		params:                          params,
   602  		filter:                          filter,
   603  		txnCache:                        common.NewUnresolvedTxnCache(),
   604  		statistics:                      NewStatistics(ctx, "mysql", opts),
   605  		metricConflictDetectDurationHis: metricConflictDetectDurationHis,
   606  		metricBucketSizeCounters:        metricBucketSizeCounters,
   607  		errCh:                           make(chan error, 1),
   608  		forceReplicate:                  replicaConfig.ForceReplicate,
   609  	}
   610  
   611  	if val, ok := opts[mark.OptCyclicConfig]; ok {
   612  		cfg := new(config.CyclicConfig)
   613  		err := cfg.Unmarshal([]byte(val))
   614  		if err != nil {
   615  			return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err)
   616  		}
   617  		sink.cyclic = cyclic.NewCyclic(cfg)
   618  
   619  		err = sink.adjustSQLMode(ctx)
   620  		if err != nil {
   621  			return nil, errors.Trace(err)
   622  		}
   623  	}
   624  
   625  	sink.execWaitNotifier = new(notify.Notifier)
   626  	sink.resolvedNotifier = new(notify.Notifier)
   627  	err = sink.createSinkWorkers(ctx)
   628  	if err != nil {
   629  		return nil, err
   630  	}
   631  
   632  	receiver, err := sink.resolvedNotifier.NewReceiver(50 * time.Millisecond)
   633  	if err != nil {
   634  		return nil, err
   635  	}
   636  	go sink.flushRowChangedEvents(ctx, receiver)
   637  
   638  	return sink, nil
   639  }
   640  
   641  func (s *mysqlSink) createSinkWorkers(ctx context.Context) error {
   642  	s.workers = make([]*mysqlSinkWorker, s.params.workerCount)
   643  	for i := range s.workers {
   644  		receiver, err := s.execWaitNotifier.NewReceiver(defaultFlushInterval)
   645  		if err != nil {
   646  			return err
   647  		}
   648  		worker := newMySQLSinkWorker(
   649  			s.params.maxTxnRow, i, s.metricBucketSizeCounters[i], receiver, s.execDMLs)
   650  		s.workers[i] = worker
   651  		go func() {
   652  			err := worker.run(ctx)
   653  			if err != nil && errors.Cause(err) != context.Canceled {
   654  				select {
   655  				case s.errCh <- err:
   656  				default:
   657  					log.Info("mysql sink receives redundant error", zap.Error(err))
   658  				}
   659  			}
   660  			worker.cleanup()
   661  		}()
   662  	}
   663  	return nil
   664  }
   665  
   666  func (s *mysqlSink) notifyAndWaitExec(ctx context.Context) {
   667  	s.broadcastFinishTxn()
   668  	s.execWaitNotifier.Notify()
   669  	done := make(chan struct{})
   670  	go func() {
   671  		s.flushSyncWg.Wait()
   672  		close(done)
   673  	}()
   674  	// This is a hack code to avoid io wait in some routine blocks others to exit.
   675  	// As the network io wait is blocked in kernel code, the goroutine is in a
   676  	// D-state that we could not even stop it by cancel the context. So if this
   677  	// scenario happens, the blocked goroutine will be leak.
   678  	select {
   679  	case <-ctx.Done():
   680  	case <-done:
   681  	}
   682  }
   683  
   684  func (s *mysqlSink) broadcastFinishTxn() {
   685  	// Note all data txn is sent via channel, the control txn must come after all
   686  	// data txns in each worker. So after worker receives the control txn, it can
   687  	// flush txns immediately and call wait group done once.
   688  	for _, worker := range s.workers {
   689  		worker.appendFinishTxn(&s.flushSyncWg)
   690  	}
   691  }
   692  
   693  func (s *mysqlSink) dispatchAndExecTxns(ctx context.Context, txnsGroup map[model.TableID][]*model.SingleTableTxn) {
   694  	nWorkers := s.params.workerCount
   695  	causality := newCausality()
   696  	rowsChIdx := 0
   697  
   698  	sendFn := func(txn *model.SingleTableTxn, keys [][]byte, idx int) {
   699  		causality.add(keys, idx)
   700  		s.workers[idx].appendTxn(ctx, txn)
   701  	}
   702  	resolveConflict := func(txn *model.SingleTableTxn) {
   703  		keys := genTxnKeys(txn)
   704  		if conflict, idx := causality.detectConflict(keys); conflict {
   705  			if idx >= 0 {
   706  				sendFn(txn, keys, idx)
   707  				return
   708  			}
   709  			s.notifyAndWaitExec(ctx)
   710  			causality.reset()
   711  		}
   712  		sendFn(txn, keys, rowsChIdx)
   713  		rowsChIdx++
   714  		rowsChIdx = rowsChIdx % nWorkers
   715  	}
   716  	h := newTxnsHeap(txnsGroup)
   717  	h.iter(func(txn *model.SingleTableTxn) {
   718  		startTime := time.Now()
   719  		resolveConflict(txn)
   720  		s.metricConflictDetectDurationHis.Observe(time.Since(startTime).Seconds())
   721  	})
   722  	s.notifyAndWaitExec(ctx)
   723  }
   724  
   725  type mysqlSinkWorker struct {
   726  	txnCh            chan *model.SingleTableTxn
   727  	maxTxnRow        int
   728  	bucket           int
   729  	execDMLs         func(context.Context, []*model.RowChangedEvent, uint64, int) error
   730  	metricBucketSize prometheus.Counter
   731  	receiver         *notify.Receiver
   732  	checkpointTs     uint64
   733  	closedCh         chan struct{}
   734  }
   735  
   736  func newMySQLSinkWorker(
   737  	maxTxnRow int,
   738  	bucket int,
   739  	metricBucketSize prometheus.Counter,
   740  	receiver *notify.Receiver,
   741  	execDMLs func(context.Context, []*model.RowChangedEvent, uint64, int) error,
   742  ) *mysqlSinkWorker {
   743  	return &mysqlSinkWorker{
   744  		txnCh:            make(chan *model.SingleTableTxn, 1024),
   745  		maxTxnRow:        maxTxnRow,
   746  		bucket:           bucket,
   747  		metricBucketSize: metricBucketSize,
   748  		execDMLs:         execDMLs,
   749  		receiver:         receiver,
   750  		closedCh:         make(chan struct{}, 1),
   751  	}
   752  }
   753  
   754  func (w *mysqlSinkWorker) appendTxn(ctx context.Context, txn *model.SingleTableTxn) {
   755  	if txn == nil {
   756  		return
   757  	}
   758  	select {
   759  	case <-ctx.Done():
   760  	case w.txnCh <- txn:
   761  	}
   762  }
   763  
   764  func (w *mysqlSinkWorker) appendFinishTxn(wg *sync.WaitGroup) {
   765  	// since worker will always fetch txns from txnCh, we don't need to worry the
   766  	// txnCh full and send is blocked.
   767  	wg.Add(1)
   768  	w.txnCh <- &model.SingleTableTxn{
   769  		FinishWg: wg,
   770  	}
   771  }
   772  
   773  func (w *mysqlSinkWorker) run(ctx context.Context) (err error) {
   774  	var (
   775  		toExecRows   []*model.RowChangedEvent
   776  		replicaID    uint64
   777  		txnNum       int
   778  		lastCommitTs uint64
   779  	)
   780  
   781  	// mark FinishWg before worker exits, all data txns can be omitted.
   782  	defer func() {
   783  		for {
   784  			select {
   785  			case txn := <-w.txnCh:
   786  				if txn.FinishWg != nil {
   787  					txn.FinishWg.Done()
   788  				}
   789  			default:
   790  				return
   791  			}
   792  		}
   793  	}()
   794  
   795  	defer func() {
   796  		if r := recover(); r != nil {
   797  			buf := make([]byte, 4096)
   798  			stackSize := runtime.Stack(buf, false)
   799  			buf = buf[:stackSize]
   800  			err = cerror.ErrMySQLWorkerPanic.GenWithStack("mysql sink concurrent execute panic, stack: %v", string(buf))
   801  			log.Error("mysql sink worker panic", zap.Reflect("r", r), zap.Stack("stack trace"))
   802  		}
   803  	}()
   804  
   805  	flushRows := func() error {
   806  		if len(toExecRows) == 0 {
   807  			return nil
   808  		}
   809  		rows := make([]*model.RowChangedEvent, len(toExecRows))
   810  		copy(rows, toExecRows)
   811  		err := w.execDMLs(ctx, rows, replicaID, w.bucket)
   812  		if err != nil {
   813  			txnNum = 0
   814  			return err
   815  		}
   816  		atomic.StoreUint64(&w.checkpointTs, lastCommitTs)
   817  		toExecRows = toExecRows[:0]
   818  		w.metricBucketSize.Add(float64(txnNum))
   819  		txnNum = 0
   820  		return nil
   821  	}
   822  
   823  	for {
   824  		select {
   825  		case <-ctx.Done():
   826  			return errors.Trace(ctx.Err())
   827  		case txn := <-w.txnCh:
   828  			if txn == nil {
   829  				return errors.Trace(flushRows())
   830  			}
   831  			if txn.FinishWg != nil {
   832  				if err := flushRows(); err != nil {
   833  					return errors.Trace(err)
   834  				}
   835  				txn.FinishWg.Done()
   836  				continue
   837  			}
   838  			if txn.ReplicaID != replicaID || len(toExecRows)+len(txn.Rows) > w.maxTxnRow {
   839  				if err := flushRows(); err != nil {
   840  					txnNum++
   841  					return errors.Trace(err)
   842  				}
   843  			}
   844  			replicaID = txn.ReplicaID
   845  			toExecRows = append(toExecRows, txn.Rows...)
   846  			lastCommitTs = txn.CommitTs
   847  			txnNum++
   848  		case <-w.receiver.C:
   849  			if err := flushRows(); err != nil {
   850  				return errors.Trace(err)
   851  			}
   852  		}
   853  	}
   854  }
   855  
   856  // cleanup waits for notification from closedCh and consumes all txns from txnCh.
   857  // The exit sequence is
   858  // 1. producer(sink.flushRowChangedEvents goroutine) of txnCh exits
   859  // 2. goroutine in 1 sends notification to closedCh of each sink worker
   860  // 3. each sink worker receives the notification from closedCh and mark FinishWg as Done
   861  func (w *mysqlSinkWorker) cleanup() {
   862  	<-w.closedCh
   863  	for {
   864  		select {
   865  		case txn := <-w.txnCh:
   866  			if txn.FinishWg != nil {
   867  				txn.FinishWg.Done()
   868  			}
   869  		default:
   870  			return
   871  		}
   872  	}
   873  }
   874  
   875  func (s *mysqlSink) Close(ctx context.Context) error {
   876  	s.execWaitNotifier.Close()
   877  	s.resolvedNotifier.Close()
   878  	err := s.db.Close()
   879  	return cerror.WrapError(cerror.ErrMySQLConnectionError, err)
   880  }
   881  
   882  func (s *mysqlSink) Barrier(ctx context.Context) error {
   883  	warnDuration := 3 * time.Minute
   884  	ticker := time.NewTicker(warnDuration)
   885  	defer ticker.Stop()
   886  	for {
   887  		select {
   888  		case <-ctx.Done():
   889  			return errors.Trace(ctx.Err())
   890  		case <-ticker.C:
   891  			log.Warn("Barrier doesn't return in time, may be stuck",
   892  				zap.Uint64("resolved-ts", atomic.LoadUint64(&s.maxResolvedTs)),
   893  				zap.Uint64("checkpoint-ts", s.checkpointTs()))
   894  		default:
   895  			maxResolvedTs := atomic.LoadUint64(&s.maxResolvedTs)
   896  			if s.checkpointTs() >= maxResolvedTs {
   897  				return nil
   898  			}
   899  			checkpointTs, err := s.FlushRowChangedEvents(ctx, maxResolvedTs)
   900  			if err != nil {
   901  				return err
   902  			}
   903  			if checkpointTs >= maxResolvedTs {
   904  				return nil
   905  			}
   906  			// short sleep to avoid cpu spin
   907  			time.Sleep(time.Second)
   908  		}
   909  	}
   910  }
   911  
   912  func (s *mysqlSink) checkpointTs() uint64 {
   913  	checkpointTs := atomic.LoadUint64(&s.resolvedTs)
   914  	for _, worker := range s.workers {
   915  		workerCheckpointTs := atomic.LoadUint64(&worker.checkpointTs)
   916  		if workerCheckpointTs < checkpointTs {
   917  			checkpointTs = workerCheckpointTs
   918  		}
   919  	}
   920  	return checkpointTs
   921  }
   922  
   923  func logDMLTxnErr(err error) error {
   924  	if isRetryableDMLError(err) {
   925  		log.Warn("execute DMLs with error, retry later", zap.Error(err))
   926  	}
   927  	return err
   928  }
   929  
   930  func isRetryableDMLError(err error) bool {
   931  	if !cerror.IsRetryableError(err) {
   932  		return false
   933  	}
   934  
   935  	errCode, ok := getSQLErrCode(err)
   936  	if !ok {
   937  		return true
   938  	}
   939  
   940  	switch errCode {
   941  	case mysql.ErrNoSuchTable, mysql.ErrBadDB:
   942  		return false
   943  	}
   944  	return true
   945  }
   946  
   947  func (s *mysqlSink) execDMLWithMaxRetries(ctx context.Context, dmls *preparedDMLs, bucket int) error {
   948  	if len(dmls.sqls) != len(dmls.values) {
   949  		log.Panic("unexpected number of sqls and values",
   950  			zap.Strings("sqls", dmls.sqls),
   951  			zap.Any("values", dmls.values))
   952  	}
   953  
   954  	return retry.Do(ctx, func() error {
   955  		failpoint.Inject("MySQLSinkTxnRandomError", func() {
   956  			failpoint.Return(logDMLTxnErr(errors.Trace(dmysql.ErrInvalidConn)))
   957  		})
   958  		failpoint.Inject("MySQLSinkHangLongTime", func() {
   959  			time.Sleep(time.Hour)
   960  		})
   961  
   962  		err := s.statistics.RecordBatchExecution(func() (int, error) {
   963  			tx, err := s.db.BeginTx(ctx, nil)
   964  			if err != nil {
   965  				return 0, logDMLTxnErr(cerror.WrapError(cerror.ErrMySQLTxnError, err))
   966  			}
   967  
   968  			for i, query := range dmls.sqls {
   969  				args := dmls.values[i]
   970  				log.Debug("exec row", zap.String("sql", query), zap.Any("args", args))
   971  				if _, err := tx.ExecContext(ctx, query, args...); err != nil {
   972  					if rbErr := tx.Rollback(); rbErr != nil {
   973  						log.Warn("failed to rollback txn", zap.Error(err))
   974  					}
   975  					return 0, logDMLTxnErr(cerror.WrapError(cerror.ErrMySQLTxnError, err))
   976  				}
   977  			}
   978  
   979  			if len(dmls.markSQL) != 0 {
   980  				log.Debug("exec row", zap.String("sql", dmls.markSQL))
   981  				if _, err := tx.ExecContext(ctx, dmls.markSQL); err != nil {
   982  					if rbErr := tx.Rollback(); rbErr != nil {
   983  						log.Warn("failed to rollback txn", zap.Error(err))
   984  					}
   985  					return 0, logDMLTxnErr(cerror.WrapError(cerror.ErrMySQLTxnError, err))
   986  				}
   987  			}
   988  
   989  			if err = tx.Commit(); err != nil {
   990  				return 0, logDMLTxnErr(cerror.WrapError(cerror.ErrMySQLTxnError, err))
   991  			}
   992  			return dmls.rowCount, nil
   993  		})
   994  		if err != nil {
   995  			return errors.Trace(err)
   996  		}
   997  		log.Debug("Exec Rows succeeded",
   998  			zap.String("changefeed", s.params.changefeedID),
   999  			zap.Int("num of Rows", dmls.rowCount),
  1000  			zap.Int("bucket", bucket))
  1001  		return nil
  1002  	}, retry.WithBackoffBaseDelay(backoffBaseDelayInMs), retry.WithBackoffMaxDelay(backoffMaxDelayInMs), retry.WithMaxTries(defaultDMLMaxRetryTime), retry.WithIsRetryableErr(isRetryableDMLError))
  1003  }
  1004  
  1005  type preparedDMLs struct {
  1006  	sqls     []string
  1007  	values   [][]interface{}
  1008  	markSQL  string
  1009  	rowCount int
  1010  }
  1011  
  1012  // prepareDMLs converts model.RowChangedEvent list to query string list and args list
  1013  func (s *mysqlSink) prepareDMLs(rows []*model.RowChangedEvent, replicaID uint64, bucket int) *preparedDMLs {
  1014  	sqls := make([]string, 0, len(rows))
  1015  	values := make([][]interface{}, 0, len(rows))
  1016  	replaces := make(map[string][][]interface{})
  1017  	rowCount := 0
  1018  	translateToInsert := s.params.enableOldValue && !s.params.safeMode
  1019  
  1020  	// flush cached batch replace or insert, to keep the sequence of DMLs
  1021  	flushCacheDMLs := func() {
  1022  		if s.params.batchReplaceEnabled && len(replaces) > 0 {
  1023  			replaceSqls, replaceValues := reduceReplace(replaces, s.params.batchReplaceSize)
  1024  			sqls = append(sqls, replaceSqls...)
  1025  			values = append(values, replaceValues...)
  1026  			replaces = make(map[string][][]interface{})
  1027  		}
  1028  	}
  1029  
  1030  	for _, row := range rows {
  1031  		var query string
  1032  		var args []interface{}
  1033  		quoteTable := quotes.QuoteSchema(row.Table.Schema, row.Table.Table)
  1034  
  1035  		// Translate to UPDATE if old value is enabled, not in safe mode and is update event
  1036  		if translateToInsert && len(row.PreColumns) != 0 && len(row.Columns) != 0 {
  1037  			flushCacheDMLs()
  1038  			query, args = prepareUpdate(quoteTable, row.PreColumns, row.Columns, s.forceReplicate)
  1039  			if query != "" {
  1040  				sqls = append(sqls, query)
  1041  				values = append(values, args)
  1042  				rowCount++
  1043  			}
  1044  			continue
  1045  		}
  1046  
  1047  		// Case for delete event or update event
  1048  		// If old value is enabled and not in safe mode,
  1049  		// update will be translated to DELETE + INSERT(or REPLACE) SQL.
  1050  		if len(row.PreColumns) != 0 {
  1051  			flushCacheDMLs()
  1052  			query, args = prepareDelete(quoteTable, row.PreColumns, s.forceReplicate)
  1053  			if query != "" {
  1054  				sqls = append(sqls, query)
  1055  				values = append(values, args)
  1056  				rowCount++
  1057  			}
  1058  		}
  1059  
  1060  		// Case for insert event or update event
  1061  		if len(row.Columns) != 0 {
  1062  			if s.params.batchReplaceEnabled {
  1063  				query, args = prepareReplace(quoteTable, row.Columns, false /* appendPlaceHolder */, translateToInsert)
  1064  				if query != "" {
  1065  					if _, ok := replaces[query]; !ok {
  1066  						replaces[query] = make([][]interface{}, 0)
  1067  					}
  1068  					replaces[query] = append(replaces[query], args)
  1069  					rowCount++
  1070  				}
  1071  			} else {
  1072  				query, args = prepareReplace(quoteTable, row.Columns, true /* appendPlaceHolder */, translateToInsert)
  1073  				sqls = append(sqls, query)
  1074  				values = append(values, args)
  1075  				if query != "" {
  1076  					sqls = append(sqls, query)
  1077  					values = append(values, args)
  1078  					rowCount++
  1079  				}
  1080  			}
  1081  		}
  1082  	}
  1083  	flushCacheDMLs()
  1084  
  1085  	dmls := &preparedDMLs{
  1086  		sqls:   sqls,
  1087  		values: values,
  1088  	}
  1089  	if s.cyclic != nil && len(rows) > 0 {
  1090  		// Write mark table with the current replica ID.
  1091  		row := rows[0]
  1092  		updateMark := s.cyclic.UdpateSourceTableCyclicMark(
  1093  			row.Table.Schema, row.Table.Table, uint64(bucket), replicaID, row.StartTs)
  1094  		dmls.markSQL = updateMark
  1095  		// rowCount is used in statistics, and for simplicity,
  1096  		// we do not count mark table rows in rowCount.
  1097  	}
  1098  	dmls.rowCount = rowCount
  1099  	return dmls
  1100  }
  1101  
  1102  func (s *mysqlSink) execDMLs(ctx context.Context, rows []*model.RowChangedEvent, replicaID uint64, bucket int) error {
  1103  	failpoint.Inject("SinkFlushDMLPanic", func() {
  1104  		time.Sleep(time.Second)
  1105  		log.Fatal("SinkFlushDMLPanic")
  1106  	})
  1107  	failpoint.Inject("MySQLSinkExecDMLError", func() {
  1108  		// Add a delay to ensure the sink worker with `MySQLSinkHangLongTime`
  1109  		// failpoint injected is executed first.
  1110  		time.Sleep(time.Second * 2)
  1111  		failpoint.Return(errors.Trace(dmysql.ErrInvalidConn))
  1112  	})
  1113  	dmls := s.prepareDMLs(rows, replicaID, bucket)
  1114  	log.Debug("prepare DMLs", zap.Any("rows", rows), zap.Strings("sqls", dmls.sqls), zap.Any("values", dmls.values))
  1115  	if err := s.execDMLWithMaxRetries(ctx, dmls, bucket); err != nil {
  1116  		log.Error("execute DMLs failed", zap.String("err", err.Error()))
  1117  		return errors.Trace(err)
  1118  	}
  1119  	return nil
  1120  }
  1121  
  1122  func prepareReplace(
  1123  	quoteTable string,
  1124  	cols []*model.Column,
  1125  	appendPlaceHolder bool,
  1126  	translateToInsert bool,
  1127  ) (string, []interface{}) {
  1128  	var builder strings.Builder
  1129  	columnNames := make([]string, 0, len(cols))
  1130  	args := make([]interface{}, 0, len(cols))
  1131  	for _, col := range cols {
  1132  		if col == nil || col.Flag.IsGeneratedColumn() {
  1133  			continue
  1134  		}
  1135  		columnNames = append(columnNames, col.Name)
  1136  		args = append(args, col.Value)
  1137  	}
  1138  	if len(args) == 0 {
  1139  		return "", nil
  1140  	}
  1141  
  1142  	colList := "(" + buildColumnList(columnNames) + ")"
  1143  	if translateToInsert {
  1144  		builder.WriteString("INSERT INTO " + quoteTable + colList + " VALUES ")
  1145  	} else {
  1146  		builder.WriteString("REPLACE INTO " + quoteTable + colList + " VALUES ")
  1147  	}
  1148  	if appendPlaceHolder {
  1149  		builder.WriteString("(" + model.HolderString(len(columnNames)) + ");")
  1150  	}
  1151  
  1152  	return builder.String(), args
  1153  }
  1154  
  1155  // reduceReplace groups SQLs with the same replace statement format, as following
  1156  // sql: `REPLACE INTO `test`.`t` (`a`,`b`) VALUES (?,?,?,?,?,?)`
  1157  // args: (1,"",2,"2",3,"")
  1158  func reduceReplace(replaces map[string][][]interface{}, batchSize int) ([]string, [][]interface{}) {
  1159  	nextHolderString := func(query string, valueNum int, last bool) string {
  1160  		query += "(" + model.HolderString(valueNum) + ")"
  1161  		if !last {
  1162  			query += ","
  1163  		}
  1164  		return query
  1165  	}
  1166  	sqls := make([]string, 0)
  1167  	args := make([][]interface{}, 0)
  1168  	for replace, vals := range replaces {
  1169  		query := replace
  1170  		cacheCount := 0
  1171  		cacheArgs := make([]interface{}, 0)
  1172  		last := false
  1173  		for i, val := range vals {
  1174  			cacheCount++
  1175  			if i == len(vals)-1 || cacheCount >= batchSize {
  1176  				last = true
  1177  			}
  1178  			query = nextHolderString(query, len(val), last)
  1179  			cacheArgs = append(cacheArgs, val...)
  1180  			if last {
  1181  				sqls = append(sqls, query)
  1182  				args = append(args, cacheArgs)
  1183  				query = replace
  1184  				cacheCount = 0
  1185  				cacheArgs = make([]interface{}, 0, len(cacheArgs))
  1186  				last = false
  1187  			}
  1188  		}
  1189  	}
  1190  	return sqls, args
  1191  }
  1192  
  1193  func prepareUpdate(quoteTable string, preCols, cols []*model.Column, forceReplicate bool) (string, []interface{}) {
  1194  	var builder strings.Builder
  1195  	builder.WriteString("UPDATE " + quoteTable + " SET ")
  1196  
  1197  	columnNames := make([]string, 0, len(cols))
  1198  	args := make([]interface{}, 0, len(cols)+len(preCols))
  1199  	for _, col := range cols {
  1200  		if col == nil || col.Flag.IsGeneratedColumn() {
  1201  			continue
  1202  		}
  1203  		columnNames = append(columnNames, col.Name)
  1204  		args = append(args, col.Value)
  1205  	}
  1206  	if len(args) == 0 {
  1207  		return "", nil
  1208  	}
  1209  	for i, column := range columnNames {
  1210  		if i == len(columnNames)-1 {
  1211  			builder.WriteString("`" + quotes.EscapeName(column) + "`=?")
  1212  		} else {
  1213  			builder.WriteString("`" + quotes.EscapeName(column) + "`=?,")
  1214  		}
  1215  	}
  1216  
  1217  	builder.WriteString(" WHERE ")
  1218  	colNames, wargs := whereSlice(preCols, forceReplicate)
  1219  	if len(wargs) == 0 {
  1220  		return "", nil
  1221  	}
  1222  	for i := 0; i < len(colNames); i++ {
  1223  		if i > 0 {
  1224  			builder.WriteString(" AND ")
  1225  		}
  1226  		if wargs[i] == nil {
  1227  			builder.WriteString(quotes.QuoteName(colNames[i]) + " IS NULL")
  1228  		} else {
  1229  			builder.WriteString(quotes.QuoteName(colNames[i]) + "=?")
  1230  			args = append(args, wargs[i])
  1231  		}
  1232  	}
  1233  	builder.WriteString(" LIMIT 1;")
  1234  	sql := builder.String()
  1235  	return sql, args
  1236  }
  1237  
  1238  func prepareDelete(quoteTable string, cols []*model.Column, forceReplicate bool) (string, []interface{}) {
  1239  	var builder strings.Builder
  1240  	builder.WriteString("DELETE FROM " + quoteTable + " WHERE ")
  1241  
  1242  	colNames, wargs := whereSlice(cols, forceReplicate)
  1243  	if len(wargs) == 0 {
  1244  		return "", nil
  1245  	}
  1246  	args := make([]interface{}, 0, len(wargs))
  1247  	for i := 0; i < len(colNames); i++ {
  1248  		if i > 0 {
  1249  			builder.WriteString(" AND ")
  1250  		}
  1251  		if wargs[i] == nil {
  1252  			builder.WriteString(quotes.QuoteName(colNames[i]) + " IS NULL")
  1253  		} else {
  1254  			builder.WriteString(quotes.QuoteName(colNames[i]) + " = ?")
  1255  			args = append(args, wargs[i])
  1256  		}
  1257  	}
  1258  	builder.WriteString(" LIMIT 1;")
  1259  	sql := builder.String()
  1260  	return sql, args
  1261  }
  1262  
  1263  func whereSlice(cols []*model.Column, forceReplicate bool) (colNames []string, args []interface{}) {
  1264  	// Try to use unique key values when available
  1265  	for _, col := range cols {
  1266  		if col == nil || !col.Flag.IsHandleKey() {
  1267  			continue
  1268  		}
  1269  		colNames = append(colNames, col.Name)
  1270  		args = append(args, col.Value)
  1271  	}
  1272  	// if no explicit row id but force replicate, use all key-values in where condition
  1273  	if len(colNames) == 0 && forceReplicate {
  1274  		colNames = make([]string, 0, len(cols))
  1275  		args = make([]interface{}, 0, len(cols))
  1276  		for _, col := range cols {
  1277  			colNames = append(colNames, col.Name)
  1278  			args = append(args, col.Value)
  1279  		}
  1280  	}
  1281  	return
  1282  }
  1283  
  1284  func isIgnorableDDLError(err error) bool {
  1285  	errCode, ok := getSQLErrCode(err)
  1286  	if !ok {
  1287  		return false
  1288  	}
  1289  	// we can get error code from:
  1290  	// infoschema's error definition: https://github.com/pingcap/tidb/blob/master/infoschema/infoschema.go
  1291  	// DDL's error definition: https://github.com/pingcap/tidb/blob/master/ddl/ddl.go
  1292  	// tidb/mysql error code definition: https://github.com/pingcap/tidb/blob/master/mysql/errcode.go
  1293  	switch errCode {
  1294  	case infoschema.ErrDatabaseExists.Code(), infoschema.ErrDatabaseNotExists.Code(), infoschema.ErrDatabaseDropExists.Code(),
  1295  		infoschema.ErrTableExists.Code(), infoschema.ErrTableNotExists.Code(), infoschema.ErrTableDropExists.Code(),
  1296  		infoschema.ErrColumnExists.Code(), infoschema.ErrColumnNotExists.Code(), infoschema.ErrIndexExists.Code(),
  1297  		infoschema.ErrKeyNotExists.Code(), tddl.ErrCantDropFieldOrKey.Code(), mysql.ErrDupKeyName, mysql.ErrSameNamePartition,
  1298  		mysql.ErrDropPartitionNonExistent, mysql.ErrMultiplePriKey:
  1299  		return true
  1300  	default:
  1301  		return false
  1302  	}
  1303  }
  1304  
  1305  func getSQLErrCode(err error) (errors.ErrCode, bool) {
  1306  	mysqlErr, ok := errors.Cause(err).(*dmysql.MySQLError)
  1307  	if !ok {
  1308  		return -1, false
  1309  	}
  1310  
  1311  	return errors.ErrCode(mysqlErr.Number), true
  1312  }
  1313  
  1314  func buildColumnList(names []string) string {
  1315  	var b strings.Builder
  1316  	for i, name := range names {
  1317  		if i > 0 {
  1318  			b.WriteString(",")
  1319  		}
  1320  		b.WriteString(quotes.QuoteName(name))
  1321  
  1322  	}
  1323  
  1324  	return b.String()
  1325  }
  1326  
  1327  // newSyncpointStore create a sink to record the syncpoint map in downstream DB for every changefeed
  1328  func newMySQLSyncpointStore(ctx context.Context, id string, sinkURI *url.URL) (SyncpointStore, error) {
  1329  	var syncDB *sql.DB
  1330  
  1331  	// todo If is neither mysql nor tidb, such as kafka, just ignore this feature.
  1332  	scheme := strings.ToLower(sinkURI.Scheme)
  1333  	if scheme != "mysql" && scheme != "tidb" && scheme != "mysql+ssl" && scheme != "tidb+ssl" {
  1334  		return nil, errors.New("can create mysql sink with unsupported scheme")
  1335  	}
  1336  	params := defaultParams.Clone()
  1337  	s := sinkURI.Query().Get("tidb-txn-mode")
  1338  	if s != "" {
  1339  		if s == "pessimistic" || s == "optimistic" {
  1340  			params.tidbTxnMode = s
  1341  		} else {
  1342  			log.Warn("invalid tidb-txn-mode, should be pessimistic or optimistic, use optimistic as default")
  1343  		}
  1344  	}
  1345  	var tlsParam string
  1346  	if sinkURI.Query().Get("ssl-ca") != "" {
  1347  		credential := security.Credential{
  1348  			CAPath:   sinkURI.Query().Get("ssl-ca"),
  1349  			CertPath: sinkURI.Query().Get("ssl-cert"),
  1350  			KeyPath:  sinkURI.Query().Get("ssl-key"),
  1351  		}
  1352  		tlsCfg, err := credential.ToTLSConfig()
  1353  		if err != nil {
  1354  			return nil, errors.Annotate(err, "fail to open MySQL connection")
  1355  		}
  1356  		name := "cdc_mysql_tls" + "syncpoint" + id
  1357  		err = dmysql.RegisterTLSConfig(name, tlsCfg)
  1358  		if err != nil {
  1359  			return nil, errors.Annotate(err, "fail to open MySQL connection")
  1360  		}
  1361  		tlsParam = "?tls=" + name
  1362  	}
  1363  	if _, ok := sinkURI.Query()["time-zone"]; ok {
  1364  		s = sinkURI.Query().Get("time-zone")
  1365  		if s == "" {
  1366  			params.timezone = ""
  1367  		} else {
  1368  			params.timezone = fmt.Sprintf(`"%s"`, s)
  1369  		}
  1370  	} else {
  1371  		tz := util.TimezoneFromCtx(ctx)
  1372  		params.timezone = fmt.Sprintf(`"%s"`, tz.String())
  1373  	}
  1374  
  1375  	// dsn format of the driver:
  1376  	// [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...&paramN=valueN]
  1377  	username := sinkURI.User.Username()
  1378  	password, _ := sinkURI.User.Password()
  1379  	port := sinkURI.Port()
  1380  	if username == "" {
  1381  		username = "root"
  1382  	}
  1383  	if port == "" {
  1384  		port = "4000"
  1385  	}
  1386  
  1387  	dsnStr := fmt.Sprintf("%s:%s@tcp(%s:%s)/%s", username, password, sinkURI.Hostname(), port, tlsParam)
  1388  	dsn, err := dmysql.ParseDSN(dsnStr)
  1389  	if err != nil {
  1390  		return nil, errors.Trace(err)
  1391  	}
  1392  
  1393  	// create test db used for parameter detection
  1394  	if dsn.Params == nil {
  1395  		dsn.Params = make(map[string]string, 1)
  1396  	}
  1397  	testDB, err := sql.Open("mysql", dsn.FormatDSN())
  1398  	if err != nil {
  1399  		return nil, errors.Annotate(
  1400  			cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection when configuring sink")
  1401  	}
  1402  	defer testDB.Close()
  1403  	dsnStr, err = configureSinkURI(ctx, dsn, params, testDB)
  1404  	if err != nil {
  1405  		return nil, errors.Trace(err)
  1406  	}
  1407  	syncDB, err = sql.Open("mysql", dsnStr)
  1408  	if err != nil {
  1409  		return nil, errors.Annotate(err, "Open database connection failed")
  1410  	}
  1411  	err = syncDB.PingContext(ctx)
  1412  	if err != nil {
  1413  		return nil, errors.Annotate(err, "fail to open MySQL connection")
  1414  	}
  1415  
  1416  	log.Info("Start mysql syncpoint sink")
  1417  	syncpointStore := &mysqlSyncpointStore{
  1418  		db: syncDB,
  1419  	}
  1420  
  1421  	return syncpointStore, nil
  1422  }
  1423  
  1424  func (s *mysqlSyncpointStore) CreateSynctable(ctx context.Context) error {
  1425  	database := mark.SchemaName
  1426  	tx, err := s.db.BeginTx(ctx, nil)
  1427  	if err != nil {
  1428  		log.Error("create sync table: begin Tx fail", zap.Error(err))
  1429  		return cerror.WrapError(cerror.ErrMySQLTxnError, err)
  1430  	}
  1431  	_, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database)
  1432  	if err != nil {
  1433  		err2 := tx.Rollback()
  1434  		if err2 != nil {
  1435  			log.Error("failed to create syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2)))
  1436  		}
  1437  		return cerror.WrapError(cerror.ErrMySQLTxnError, err)
  1438  	}
  1439  	_, err = tx.Exec("USE " + database)
  1440  	if err != nil {
  1441  		err2 := tx.Rollback()
  1442  		if err2 != nil {
  1443  			log.Error("failed to create syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2)))
  1444  		}
  1445  		return cerror.WrapError(cerror.ErrMySQLTxnError, err)
  1446  	}
  1447  	_, err = tx.Exec("CREATE TABLE  IF NOT EXISTS " + syncpointTableName + " (cf varchar(255),primary_ts varchar(18),secondary_ts varchar(18),PRIMARY KEY ( `cf`, `primary_ts` ) )")
  1448  	if err != nil {
  1449  		err2 := tx.Rollback()
  1450  		if err2 != nil {
  1451  			log.Error("failed to create syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2)))
  1452  		}
  1453  		return cerror.WrapError(cerror.ErrMySQLTxnError, err)
  1454  	}
  1455  	err = tx.Commit()
  1456  	return cerror.WrapError(cerror.ErrMySQLTxnError, err)
  1457  }
  1458  
  1459  func (s *mysqlSyncpointStore) SinkSyncpoint(ctx context.Context, id string, checkpointTs uint64) error {
  1460  	tx, err := s.db.BeginTx(ctx, nil)
  1461  	if err != nil {
  1462  		log.Error("sync table: begin Tx fail", zap.Error(err))
  1463  		return cerror.WrapError(cerror.ErrMySQLTxnError, err)
  1464  	}
  1465  	row := tx.QueryRow("select @@tidb_current_ts")
  1466  	var secondaryTs string
  1467  	err = row.Scan(&secondaryTs)
  1468  	if err != nil {
  1469  		log.Info("sync table: get tidb_current_ts err")
  1470  		err2 := tx.Rollback()
  1471  		if err2 != nil {
  1472  			log.Error("failed to write syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2)))
  1473  		}
  1474  		return cerror.WrapError(cerror.ErrMySQLTxnError, err)
  1475  	}
  1476  	_, err = tx.Exec("insert ignore into "+mark.SchemaName+"."+syncpointTableName+"(cf, primary_ts, secondary_ts) VALUES (?,?,?)", id, checkpointTs, secondaryTs)
  1477  	if err != nil {
  1478  		err2 := tx.Rollback()
  1479  		if err2 != nil {
  1480  			log.Error("failed to write syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err2)))
  1481  		}
  1482  		return cerror.WrapError(cerror.ErrMySQLTxnError, err)
  1483  	}
  1484  	err = tx.Commit()
  1485  	return cerror.WrapError(cerror.ErrMySQLTxnError, err)
  1486  }
  1487  
  1488  func (s *mysqlSyncpointStore) Close() error {
  1489  	err := s.db.Close()
  1490  	return cerror.WrapError(cerror.ErrMySQLConnectionError, err)
  1491  }