github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/loader/checkpoint.go (about)

     1  // Copyright 2019 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 loader
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  
    20  	"github.com/pingcap/tidb/pkg/util/dbutil"
    21  	"github.com/pingcap/tiflow/dm/pkg/conn"
    22  	tcontext "github.com/pingcap/tiflow/dm/pkg/context"
    23  	"github.com/pingcap/tiflow/dm/pkg/cputil"
    24  	"github.com/pingcap/tiflow/dm/pkg/log"
    25  	"github.com/pingcap/tiflow/dm/pkg/terror"
    26  	"go.uber.org/zap"
    27  )
    28  
    29  type lightingLoadStatus int
    30  
    31  const (
    32  	lightningStatusInit lightingLoadStatus = iota
    33  	lightningStatusRunning
    34  	lightningStatusFinished
    35  )
    36  
    37  func (s lightingLoadStatus) String() string {
    38  	switch s {
    39  	case lightningStatusInit:
    40  		return "init"
    41  	case lightningStatusRunning:
    42  		return "running"
    43  	case lightningStatusFinished:
    44  		return "finished"
    45  	default:
    46  		panic(fmt.Sprintf("unknown lightning load stauts '%d'", s))
    47  	}
    48  }
    49  
    50  func parseLightningLoadStatus(s string) lightingLoadStatus {
    51  	switch s {
    52  	case "running":
    53  		return lightningStatusRunning
    54  	case "finished":
    55  		return lightningStatusFinished
    56  	case "init":
    57  		return lightningStatusInit
    58  	default:
    59  		log.L().Warn("unknown lightning load status, will fallback to init", zap.String("status", s))
    60  		return lightningStatusInit
    61  	}
    62  }
    63  
    64  type LightningCheckpointList struct {
    65  	db         *conn.BaseDB
    66  	schema     string
    67  	tableName  string
    68  	taskName   string
    69  	sourceName string
    70  	logger     log.Logger
    71  }
    72  
    73  func NewLightningCheckpointList(
    74  	db *conn.BaseDB,
    75  	taskName string,
    76  	sourceName string,
    77  	metaSchema string,
    78  	logger log.Logger,
    79  ) *LightningCheckpointList {
    80  	return &LightningCheckpointList{
    81  		db:         db,
    82  		schema:     dbutil.ColumnName(metaSchema),
    83  		tableName:  dbutil.TableName(metaSchema, cputil.LightningCheckpoint(taskName)),
    84  		taskName:   taskName,
    85  		sourceName: sourceName,
    86  		logger:     logger.WithFields(zap.String("component", "lightning checkpoint database list")),
    87  	}
    88  }
    89  
    90  func (cp *LightningCheckpointList) Prepare(ctx context.Context) error {
    91  	connection, err := cp.db.GetBaseConn(ctx)
    92  	if err != nil {
    93  		return terror.WithScope(terror.Annotate(err, "initialize connection when prepare"), terror.ScopeDownstream)
    94  	}
    95  	defer cp.db.ForceCloseConnWithoutErr(connection)
    96  
    97  	createSchema := fmt.Sprintf("CREATE SCHEMA IF NOT EXISTS %s", cp.schema)
    98  	tctx := tcontext.NewContext(ctx, log.With(zap.String("job", "lightning-checkpoint")))
    99  	_, err = connection.ExecuteSQL(tctx, nil, "lightning-checkpoint", []string{createSchema})
   100  	if err != nil {
   101  		return err
   102  	}
   103  	createTable := `CREATE TABLE IF NOT EXISTS %s (
   104  		task_name varchar(255) NOT NULL,
   105  		source_name varchar(255) NOT NULL,
   106  		status varchar(10) NOT NULL DEFAULT 'init' COMMENT 'init,running,finished',
   107  		PRIMARY KEY (task_name, source_name)
   108  	);
   109  `
   110  	sql2 := fmt.Sprintf(createTable, cp.tableName)
   111  	_, err = connection.ExecuteSQL(tctx, nil, "lightning-checkpoint", []string{sql2})
   112  	return terror.WithScope(err, terror.ScopeDownstream)
   113  }
   114  
   115  func (cp *LightningCheckpointList) RegisterCheckPoint(ctx context.Context) error {
   116  	connection, err := cp.db.GetBaseConn(ctx)
   117  	if err != nil {
   118  		return terror.WithScope(terror.Annotate(err, "initialize connection"), terror.ScopeDownstream)
   119  	}
   120  	defer cp.db.ForceCloseConnWithoutErr(connection)
   121  
   122  	sql := fmt.Sprintf("INSERT IGNORE INTO %s (`task_name`, `source_name`) VALUES (?, ?)", cp.tableName)
   123  	cp.logger.Info("initial checkpoint record",
   124  		zap.String("task", cp.taskName),
   125  		zap.String("source", cp.sourceName))
   126  	args := []interface{}{cp.taskName, cp.sourceName}
   127  	tctx := tcontext.NewContext(ctx, log.With(zap.String("job", "lightning-checkpoint")))
   128  	_, err = connection.ExecuteSQL(tctx, nil, "lightning-checkpoint", []string{sql}, args)
   129  	if err != nil {
   130  		return terror.WithScope(terror.Annotate(err, "initialize checkpoint"), terror.ScopeDownstream)
   131  	}
   132  	return nil
   133  }
   134  
   135  func (cp *LightningCheckpointList) UpdateStatus(ctx context.Context, status lightingLoadStatus) error {
   136  	connection, err := cp.db.GetBaseConn(ctx)
   137  	if err != nil {
   138  		return terror.WithScope(terror.Annotate(err, "initialize connection"), terror.ScopeDownstream)
   139  	}
   140  	defer cp.db.ForceCloseConnWithoutErr(connection)
   141  
   142  	sql := fmt.Sprintf("UPDATE %s set status = ? WHERE `task_name` = ? AND `source_name` = ?", cp.tableName)
   143  	cp.logger.Info("update lightning loader status",
   144  		zap.String("task", cp.taskName), zap.String("source", cp.sourceName),
   145  		zap.Stringer("status", status))
   146  	tctx := tcontext.NewContext(ctx, log.With(zap.String("job", "lightning-checkpoint")))
   147  	_, err = connection.ExecuteSQL(tctx, nil, "lightning-checkpoint", []string{sql},
   148  		[]interface{}{status.String(), cp.taskName, cp.sourceName})
   149  	if err != nil {
   150  		return terror.WithScope(terror.Annotate(err, "update lightning status"), terror.ScopeDownstream)
   151  	}
   152  	return nil
   153  }
   154  
   155  func (cp *LightningCheckpointList) taskStatus(ctx context.Context) (lightingLoadStatus, error) {
   156  	connection, err := cp.db.GetBaseConn(ctx)
   157  	if err != nil {
   158  		return lightningStatusInit, terror.WithScope(terror.Annotate(err, "initialize connection"), terror.ScopeDownstream)
   159  	}
   160  	defer cp.db.ForceCloseConnWithoutErr(connection)
   161  
   162  	query := fmt.Sprintf("SELECT status FROM %s WHERE `task_name` = ? AND `source_name` = ?", cp.tableName)
   163  	tctx := tcontext.NewContext(ctx, log.With(zap.String("job", "lightning-checkpoint")))
   164  	// nolint:rowserrcheck
   165  	rows, err := connection.QuerySQL(tctx, query, cp.taskName, cp.sourceName)
   166  	if err != nil {
   167  		return lightningStatusInit, err
   168  	}
   169  	defer rows.Close()
   170  	if rows.Next() {
   171  		var status string
   172  		if err = rows.Scan(&status); err != nil {
   173  			return lightningStatusInit, terror.WithScope(err, terror.ScopeDownstream)
   174  		}
   175  		return parseLightningLoadStatus(status), nil
   176  	}
   177  	// status row doesn't exist, return default value
   178  	return lightningStatusInit, nil
   179  }
   180  
   181  // Close implements CheckPoint.Close.
   182  func (cp *LightningCheckpointList) Close() {
   183  	if err := cp.db.Close(); err != nil {
   184  		cp.logger.Error("close checkpoint list db", log.ShortError(err))
   185  	}
   186  }