github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/ddlsink/mysql/mysql_ddl_sink.go (about)

     1  // Copyright 2022 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 mysql
    15  
    16  import (
    17  	"context"
    18  	"database/sql"
    19  	"fmt"
    20  	"net/url"
    21  	"time"
    22  
    23  	cerrors "github.com/pingcap/errors"
    24  	"github.com/pingcap/failpoint"
    25  	"github.com/pingcap/log"
    26  	timodel "github.com/pingcap/tidb/pkg/parser/model"
    27  	"github.com/pingcap/tiflow/cdc/model"
    28  	"github.com/pingcap/tiflow/cdc/sink/ddlsink"
    29  	"github.com/pingcap/tiflow/cdc/sink/metrics"
    30  	"github.com/pingcap/tiflow/pkg/config"
    31  	"github.com/pingcap/tiflow/pkg/errors"
    32  	cerror "github.com/pingcap/tiflow/pkg/errors"
    33  	"github.com/pingcap/tiflow/pkg/errorutil"
    34  	"github.com/pingcap/tiflow/pkg/quotes"
    35  	"github.com/pingcap/tiflow/pkg/retry"
    36  	"github.com/pingcap/tiflow/pkg/sink"
    37  	pmysql "github.com/pingcap/tiflow/pkg/sink/mysql"
    38  	"go.uber.org/zap"
    39  )
    40  
    41  const (
    42  	defaultDDLMaxRetry uint64 = 20
    43  
    44  	// networkDriftDuration is used to construct a context timeout for database operations.
    45  	networkDriftDuration = 5 * time.Second
    46  )
    47  
    48  // GetDBConnImpl is the implementation of pmysql.Factory.
    49  // Exported for testing.
    50  var GetDBConnImpl pmysql.Factory = pmysql.CreateMySQLDBConn
    51  
    52  // Assert Sink implementation
    53  var _ ddlsink.Sink = (*DDLSink)(nil)
    54  
    55  // DDLSink is a sink that writes DDL events to MySQL.
    56  type DDLSink struct {
    57  	// id indicates which processor (changefeed) this sink belongs to.
    58  	id model.ChangeFeedID
    59  	// db is the database connection.
    60  	db  *sql.DB
    61  	cfg *pmysql.Config
    62  	// statistics is the statistics of this sink.
    63  	// We use it to record the DDL count.
    64  	statistics *metrics.Statistics
    65  }
    66  
    67  // NewDDLSink creates a new DDLSink.
    68  func NewDDLSink(
    69  	ctx context.Context,
    70  	changefeedID model.ChangeFeedID,
    71  	sinkURI *url.URL,
    72  	replicaConfig *config.ReplicaConfig,
    73  ) (*DDLSink, error) {
    74  	cfg := pmysql.NewConfig()
    75  	err := cfg.Apply(config.GetGlobalServerConfig().TZ, changefeedID, sinkURI, replicaConfig)
    76  	if err != nil {
    77  		return nil, err
    78  	}
    79  
    80  	dsnStr, err := pmysql.GenerateDSN(ctx, sinkURI, cfg, GetDBConnImpl)
    81  	if err != nil {
    82  		return nil, err
    83  	}
    84  
    85  	db, err := GetDBConnImpl(ctx, dsnStr)
    86  	if err != nil {
    87  		return nil, err
    88  	}
    89  
    90  	cfg.IsTiDB, err = pmysql.CheckIsTiDB(ctx, db)
    91  	if err != nil {
    92  		return nil, err
    93  	}
    94  
    95  	cfg.IsWriteSourceExisted, err = pmysql.CheckIfBDRModeIsSupported(ctx, db)
    96  	if err != nil {
    97  		return nil, err
    98  	}
    99  
   100  	m := &DDLSink{
   101  		id:         changefeedID,
   102  		db:         db,
   103  		cfg:        cfg,
   104  		statistics: metrics.NewStatistics(ctx, changefeedID, sink.TxnSink),
   105  	}
   106  
   107  	log.Info("MySQL DDL sink is created",
   108  		zap.String("namespace", m.id.Namespace),
   109  		zap.String("changefeed", m.id.ID))
   110  	return m, nil
   111  }
   112  
   113  // WriteDDLEvent writes a DDL event to the mysql database.
   114  func (m *DDLSink) WriteDDLEvent(ctx context.Context, ddl *model.DDLEvent) error {
   115  	if ddl.Type == timodel.ActionAddIndex && m.cfg.IsTiDB {
   116  		return m.asyncExecAddIndexDDLIfTimeout(ctx, ddl)
   117  	}
   118  	return m.execDDLWithMaxRetries(ctx, ddl)
   119  }
   120  
   121  func (m *DDLSink) execDDLWithMaxRetries(ctx context.Context, ddl *model.DDLEvent) error {
   122  	return retry.Do(ctx, func() error {
   123  		err := m.statistics.RecordDDLExecution(func() error { return m.execDDL(ctx, ddl) })
   124  		if err != nil {
   125  			if errorutil.IsIgnorableMySQLDDLError(err) {
   126  				// NOTE: don't change the log, some tests depend on it.
   127  				log.Info("Execute DDL failed, but error can be ignored",
   128  					zap.Uint64("startTs", ddl.StartTs), zap.String("ddl", ddl.Query),
   129  					zap.String("namespace", m.id.Namespace),
   130  					zap.String("changefeed", m.id.ID),
   131  					zap.Error(err))
   132  				// If the error is ignorable, we will ignore the error directly.
   133  				return nil
   134  			}
   135  			log.Warn("Execute DDL with error, retry later",
   136  				zap.Uint64("startTs", ddl.StartTs), zap.String("ddl", ddl.Query),
   137  				zap.String("namespace", m.id.Namespace),
   138  				zap.String("changefeed", m.id.ID),
   139  				zap.Error(err))
   140  			return err
   141  		}
   142  		return nil
   143  	}, retry.WithBackoffBaseDelay(pmysql.BackoffBaseDelay.Milliseconds()),
   144  		retry.WithBackoffMaxDelay(pmysql.BackoffMaxDelay.Milliseconds()),
   145  		retry.WithMaxTries(defaultDDLMaxRetry),
   146  		retry.WithIsRetryableErr(errorutil.IsRetryableDDLError))
   147  }
   148  
   149  // isReorgOrPartitionDDL returns true if given ddl type is reorg ddl or
   150  // partition ddl.
   151  func isReorgOrPartitionDDL(t timodel.ActionType) bool {
   152  	// partition related ddl
   153  	return t == timodel.ActionAddTablePartition ||
   154  		t == timodel.ActionExchangeTablePartition ||
   155  		t == timodel.ActionReorganizePartition ||
   156  		// reorg ddls
   157  		t == timodel.ActionAddPrimaryKey ||
   158  		t == timodel.ActionAddIndex ||
   159  		t == timodel.ActionModifyColumn ||
   160  		// following ddls can be fast when the downstream is TiDB, we must
   161  		// still take them into consideration to ensure compatibility with all
   162  		// MySQL-compatible databases.
   163  		t == timodel.ActionAddColumn ||
   164  		t == timodel.ActionAddColumns ||
   165  		t == timodel.ActionDropColumn ||
   166  		t == timodel.ActionDropColumns
   167  }
   168  
   169  func (m *DDLSink) execDDL(pctx context.Context, ddl *model.DDLEvent) error {
   170  	ctx := pctx
   171  	// When executing Reorg and Partition DDLs in TiDB, there is no timeout
   172  	// mechanism by default. Instead, the system will wait for the DDL operation
   173  	// to be executed or completed before proceeding.
   174  	if !isReorgOrPartitionDDL(ddl.Type) {
   175  		writeTimeout, _ := time.ParseDuration(m.cfg.WriteTimeout)
   176  		writeTimeout += networkDriftDuration
   177  		var cancelFunc func()
   178  		ctx, cancelFunc = context.WithTimeout(pctx, writeTimeout)
   179  		defer cancelFunc()
   180  	}
   181  
   182  	shouldSwitchDB := needSwitchDB(ddl)
   183  
   184  	failpoint.Inject("MySQLSinkExecDDLDelay", func() {
   185  		select {
   186  		case <-ctx.Done():
   187  			failpoint.Return(ctx.Err())
   188  		case <-time.After(time.Hour):
   189  		}
   190  		failpoint.Return(nil)
   191  	})
   192  
   193  	start := time.Now()
   194  	log.Info("Start exec DDL", zap.String("DDL", ddl.Query), zap.Uint64("commitTs", ddl.CommitTs),
   195  		zap.String("namespace", m.id.Namespace), zap.String("changefeed", m.id.ID))
   196  	tx, err := m.db.BeginTx(ctx, nil)
   197  	if err != nil {
   198  		return err
   199  	}
   200  
   201  	if shouldSwitchDB {
   202  		_, err = tx.ExecContext(ctx, "USE "+quotes.QuoteName(ddl.TableInfo.TableName.Schema)+";")
   203  		if err != nil {
   204  			if rbErr := tx.Rollback(); rbErr != nil {
   205  				log.Error("Failed to rollback", zap.String("namespace", m.id.Namespace),
   206  					zap.String("changefeed", m.id.ID), zap.Error(err))
   207  			}
   208  			return err
   209  		}
   210  	}
   211  
   212  	// we try to set cdc write source for the ddl
   213  	if err = pmysql.SetWriteSource(pctx, m.cfg, tx); err != nil {
   214  		if rbErr := tx.Rollback(); rbErr != nil {
   215  			if errors.Cause(rbErr) != context.Canceled {
   216  				log.Error("Failed to rollback",
   217  					zap.String("namespace", m.id.Namespace),
   218  					zap.String("changefeed", m.id.ID), zap.Error(err))
   219  			}
   220  		}
   221  		return err
   222  	}
   223  
   224  	if _, err = tx.ExecContext(ctx, ddl.Query); err != nil {
   225  		if rbErr := tx.Rollback(); rbErr != nil {
   226  			log.Error("Failed to rollback", zap.String("sql", ddl.Query),
   227  				zap.String("namespace", m.id.Namespace),
   228  				zap.String("changefeed", m.id.ID), zap.Error(err))
   229  		}
   230  		return err
   231  	}
   232  
   233  	if err = tx.Commit(); err != nil {
   234  		log.Error("Failed to exec DDL", zap.String("sql", ddl.Query),
   235  			zap.Duration("duration", time.Since(start)),
   236  			zap.String("namespace", m.id.Namespace),
   237  			zap.String("changefeed", m.id.ID), zap.Error(err))
   238  		return cerror.WrapError(cerror.ErrMySQLTxnError, cerrors.WithMessage(err, fmt.Sprintf("Query info: %s; ", ddl.Query)))
   239  	}
   240  
   241  	log.Info("Exec DDL succeeded", zap.String("sql", ddl.Query),
   242  		zap.Duration("duration", time.Since(start)),
   243  		zap.String("namespace", m.id.Namespace),
   244  		zap.String("changefeed", m.id.ID))
   245  	return nil
   246  }
   247  
   248  func needSwitchDB(ddl *model.DDLEvent) bool {
   249  	if len(ddl.TableInfo.TableName.Schema) == 0 {
   250  		return false
   251  	}
   252  	if ddl.Type == timodel.ActionCreateSchema || ddl.Type == timodel.ActionDropSchema {
   253  		return false
   254  	}
   255  	return true
   256  }
   257  
   258  // WriteCheckpointTs does nothing.
   259  func (m *DDLSink) WriteCheckpointTs(_ context.Context, _ uint64, _ []*model.TableInfo) error {
   260  	// Only for RowSink for now.
   261  	return nil
   262  }
   263  
   264  // Close closes the database connection.
   265  func (m *DDLSink) Close() {
   266  	if m.statistics != nil {
   267  		m.statistics.Close()
   268  	}
   269  	if m.db != nil {
   270  		if err := m.db.Close(); err != nil {
   271  			log.Warn("MySQL ddl sink close db wit error",
   272  				zap.String("namespace", m.id.Namespace),
   273  				zap.String("changefeed", m.id.ID),
   274  				zap.Error(err))
   275  		}
   276  	}
   277  }
   278  
   279  // asyncExecAddIndexDDLIfTimeout executes ddl in async mode.
   280  // this function only works in TiDB, because TiDB will save ddl jobs
   281  // and execute them asynchronously even if ticdc crashed.
   282  func (m *DDLSink) asyncExecAddIndexDDLIfTimeout(ctx context.Context, ddl *model.DDLEvent) error {
   283  	done := make(chan error, 1)
   284  	// wait for 2 seconds at most
   285  	tick := time.NewTimer(2 * time.Second)
   286  	defer tick.Stop()
   287  	log.Info("async exec add index ddl start",
   288  		zap.String("changefeedID", m.id.String()),
   289  		zap.Uint64("commitTs", ddl.CommitTs),
   290  		zap.String("ddl", ddl.Query))
   291  	go func() {
   292  		if err := m.execDDLWithMaxRetries(ctx, ddl); err != nil {
   293  			log.Error("async exec add index ddl failed",
   294  				zap.String("changefeedID", m.id.String()),
   295  				zap.Uint64("commitTs", ddl.CommitTs),
   296  				zap.String("ddl", ddl.Query))
   297  			done <- err
   298  			return
   299  		}
   300  		log.Info("async exec add index ddl done",
   301  			zap.String("changefeedID", m.id.String()),
   302  			zap.Uint64("commitTs", ddl.CommitTs),
   303  			zap.String("ddl", ddl.Query))
   304  		done <- nil
   305  	}()
   306  
   307  	select {
   308  	case <-ctx.Done():
   309  		// if the ddl is canceled, we just return nil, if the ddl is not received by tidb,
   310  		// the downstream ddl is lost, because the checkpoint ts is forwarded.
   311  		log.Info("async add index ddl exits as canceled",
   312  			zap.String("changefeedID", m.id.String()),
   313  			zap.Uint64("commitTs", ddl.CommitTs),
   314  			zap.String("ddl", ddl.Query))
   315  		return nil
   316  	case err := <-done:
   317  		// if the ddl is executed within 2 seconds, we just return the result to the caller.
   318  		return err
   319  	case <-tick.C:
   320  		// if the ddl is still running, we just return nil,
   321  		// then if the ddl is failed, the downstream ddl is lost.
   322  		// because the checkpoint ts is forwarded.
   323  		log.Info("async add index ddl is still running",
   324  			zap.String("changefeedID", m.id.String()),
   325  			zap.Uint64("commitTs", ddl.CommitTs),
   326  			zap.String("ddl", ddl.Query))
   327  		return nil
   328  	}
   329  }