github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/syncpointstore/mysql_syncpoint_store.go (about)

     1  // Copyright 2021 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 syncpointstore
    15  
    16  import (
    17  	"context"
    18  	"database/sql"
    19  	"fmt"
    20  	"net/url"
    21  	"time"
    22  
    23  	"github.com/pingcap/errors"
    24  	"github.com/pingcap/log"
    25  	"github.com/pingcap/tiflow/cdc/model"
    26  	"github.com/pingcap/tiflow/pkg/config"
    27  	cerror "github.com/pingcap/tiflow/pkg/errors"
    28  	"github.com/pingcap/tiflow/pkg/errorutil"
    29  	"github.com/pingcap/tiflow/pkg/filter"
    30  	"github.com/pingcap/tiflow/pkg/sink/mysql"
    31  	"go.uber.org/zap"
    32  )
    33  
    34  type mysqlSyncPointStore struct {
    35  	db                     *sql.DB
    36  	clusterID              string
    37  	syncPointRetention     time.Duration
    38  	lastCleanSyncPointTime time.Time
    39  }
    40  
    41  // newSyncPointStore create a sink to record the syncPoint map in downstream DB for every changefeed
    42  func newMySQLSyncPointStore(
    43  	ctx context.Context,
    44  	id model.ChangeFeedID,
    45  	sinkURI *url.URL,
    46  	syncPointRetention time.Duration,
    47  ) (SyncPointStore, error) {
    48  	cfg := mysql.NewConfig()
    49  	err := cfg.Apply(config.GetGlobalServerConfig().TZ, id, sinkURI, config.GetDefaultReplicaConfig())
    50  	if err != nil {
    51  		return nil, err
    52  	}
    53  	getTestDb := func(ctx context.Context, dsnStr string) (*sql.DB, error) {
    54  		testDB, err := sql.Open("mysql", dsnStr)
    55  		if err != nil {
    56  			return nil, err
    57  		}
    58  
    59  		return testDB, nil
    60  	}
    61  	dsnStr, err := mysql.GenerateDSN(ctx, sinkURI, cfg, getTestDb)
    62  	if err != nil {
    63  		return nil, errors.Trace(err)
    64  	}
    65  	syncDB, err := sql.Open("mysql", dsnStr)
    66  	if err != nil {
    67  		return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection")
    68  	}
    69  	err = syncDB.PingContext(ctx)
    70  	if err != nil {
    71  		return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection")
    72  	}
    73  
    74  	log.Info("Start mysql syncpoint sink", zap.String("changefeed", id.String()))
    75  
    76  	return &mysqlSyncPointStore{
    77  		db:                     syncDB,
    78  		clusterID:              config.GetGlobalServerConfig().ClusterID,
    79  		syncPointRetention:     syncPointRetention,
    80  		lastCleanSyncPointTime: time.Now(),
    81  	}, nil
    82  }
    83  
    84  func (s *mysqlSyncPointStore) CreateSyncTable(ctx context.Context) error {
    85  	database := filter.TiCDCSystemSchema
    86  	tx, err := s.db.BeginTx(ctx, nil)
    87  	if err != nil {
    88  		log.Error("create sync table: begin Tx fail", zap.Error(err))
    89  		return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "create sync table: begin Tx fail;"))
    90  	}
    91  	_, err = tx.Exec("CREATE DATABASE IF NOT EXISTS " + database)
    92  	if err != nil {
    93  		err2 := tx.Rollback()
    94  		if err2 != nil {
    95  			log.Error("failed to create syncpoint table", zap.Error(err2))
    96  		}
    97  		return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;"))
    98  	}
    99  	_, err = tx.Exec("USE " + database)
   100  	if err != nil {
   101  		err2 := tx.Rollback()
   102  		if err2 != nil {
   103  			log.Error("failed to create syncpoint table", zap.Error(err2))
   104  		}
   105  		return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;"))
   106  	}
   107  	query := `CREATE TABLE IF NOT EXISTS %s
   108  	(
   109  		ticdc_cluster_id varchar (255),
   110  		changefeed varchar(255),
   111  		primary_ts varchar(18),
   112  		secondary_ts varchar(18),
   113  		created_at timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP,
   114  		INDEX (created_at),
   115  		PRIMARY KEY (changefeed, primary_ts)
   116  	);`
   117  	query = fmt.Sprintf(query, filter.SyncPointTable)
   118  	_, err = tx.Exec(query)
   119  	if err != nil {
   120  		err2 := tx.Rollback()
   121  		if err2 != nil {
   122  			log.Error("failed to create syncpoint table", zap.Error(err2))
   123  		}
   124  		return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;"))
   125  	}
   126  	err = tx.Commit()
   127  	return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to create syncpoint table;"))
   128  }
   129  
   130  func (s *mysqlSyncPointStore) SinkSyncPoint(ctx context.Context,
   131  	id model.ChangeFeedID,
   132  	checkpointTs uint64,
   133  ) error {
   134  	tx, err := s.db.BeginTx(ctx, nil)
   135  	if err != nil {
   136  		log.Error("sync table: begin Tx fail", zap.Error(err))
   137  		return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "sync table: begin Tx fail;"))
   138  	}
   139  	row := tx.QueryRow("select @@tidb_current_ts")
   140  	var secondaryTs string
   141  	err = row.Scan(&secondaryTs)
   142  	if err != nil {
   143  		log.Info("sync table: get tidb_current_ts err", zap.String("changefeed", id.String()))
   144  		err2 := tx.Rollback()
   145  		if err2 != nil {
   146  			log.Error("failed to write syncpoint table", zap.Error(err))
   147  		}
   148  		return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;"))
   149  	}
   150  	// insert ts map
   151  	query := "insert ignore into " + filter.TiCDCSystemSchema + "." + filter.SyncPointTable +
   152  		"(ticdc_cluster_id, changefeed, primary_ts, secondary_ts) VALUES (?,?,?,?)"
   153  	_, err = tx.Exec(query, s.clusterID, id.ID, checkpointTs, secondaryTs)
   154  	if err != nil {
   155  		err2 := tx.Rollback()
   156  		if err2 != nil {
   157  			log.Error("failed to write syncpoint table", zap.Error(err2))
   158  		}
   159  		return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;"))
   160  	}
   161  
   162  	// set global tidb_external_ts to secondary ts
   163  	// TiDB supports tidb_external_ts system variable since v6.4.0.
   164  	query = fmt.Sprintf("set global tidb_external_ts = %s", secondaryTs)
   165  	_, err = tx.Exec(query)
   166  	if err != nil {
   167  		if errorutil.IsSyncPointIgnoreError(err) {
   168  			// TODO(dongmen): to confirm if we need to log this error.
   169  			log.Warn("set global external ts failed, ignore this error", zap.Error(err))
   170  		} else {
   171  			err2 := tx.Rollback()
   172  			if err2 != nil {
   173  				log.Error("failed to write syncpoint table", zap.Error(err2))
   174  			}
   175  			return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;"))
   176  		}
   177  	}
   178  
   179  	// clean stale ts map in downstream
   180  	if time.Since(s.lastCleanSyncPointTime) >= s.syncPointRetention {
   181  		query = fmt.Sprintf(
   182  			"DELETE IGNORE FROM "+
   183  				filter.TiCDCSystemSchema+"."+
   184  				filter.SyncPointTable+
   185  				" WHERE ticdc_cluster_id = '%s' and changefeed = '%s' and created_at < (NOW() - INTERVAL %.2f SECOND)",
   186  			s.clusterID,
   187  			id.ID,
   188  			s.syncPointRetention.Seconds())
   189  		_, err = tx.Exec(query)
   190  		if err != nil {
   191  			// It is ok to ignore the error, since it will not affect the correctness of the system,
   192  			// and no any business logic depends on this behavior, so we just log the error.
   193  			log.Error("failed to clean syncpoint table", zap.Error(cerror.WrapError(cerror.ErrMySQLTxnError, err)))
   194  		} else {
   195  			s.lastCleanSyncPointTime = time.Now()
   196  		}
   197  	}
   198  
   199  	err = tx.Commit()
   200  	return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, "failed to write syncpoint table;"))
   201  }
   202  
   203  func (s *mysqlSyncPointStore) Close() error {
   204  	err := s.db.Close()
   205  	return cerror.WrapError(cerror.ErrMySQLConnectionError, err)
   206  }