github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/redo/reader/reader.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 reader
    15  
    16  import (
    17  	"container/heap"
    18  	"context"
    19  	"io"
    20  	"net/url"
    21  	"os"
    22  	"strings"
    23  	"time"
    24  
    25  	"github.com/pingcap/log"
    26  	"github.com/pingcap/tiflow/cdc/model"
    27  	"github.com/pingcap/tiflow/cdc/redo/common"
    28  	"github.com/pingcap/tiflow/pkg/errors"
    29  	"github.com/pingcap/tiflow/pkg/redo"
    30  	"github.com/pingcap/tiflow/pkg/sink/mysql"
    31  	"github.com/pingcap/tiflow/pkg/util"
    32  	"go.uber.org/multierr"
    33  	"go.uber.org/zap"
    34  	"golang.org/x/sync/errgroup"
    35  )
    36  
    37  const (
    38  	emitBatch             = mysql.DefaultMaxTxnRow
    39  	defaultReaderChanSize = mysql.DefaultWorkerCount * emitBatch
    40  	maxTotalMemoryUsage   = 80.0
    41  	maxWaitDuration       = time.Minute * 2
    42  )
    43  
    44  // RedoLogReader is a reader abstraction for redo log storage layer
    45  type RedoLogReader interface {
    46  	// Run read and decode redo logs in background.
    47  	Run(ctx context.Context) error
    48  	// ReadNextRow read one row event from redo logs.
    49  	ReadNextRow(ctx context.Context) (*model.RowChangedEvent, error)
    50  	// ReadNextDDL read one ddl event from redo logs.
    51  	ReadNextDDL(ctx context.Context) (*model.DDLEvent, error)
    52  	// ReadMeta reads meta from redo logs and returns the latest checkpointTs and resolvedTs
    53  	ReadMeta(ctx context.Context) (checkpointTs, resolvedTs uint64, err error)
    54  }
    55  
    56  // NewRedoLogReader creates a new redo log reader
    57  func NewRedoLogReader(
    58  	ctx context.Context, storageType string, cfg *LogReaderConfig,
    59  ) (rd RedoLogReader, err error) {
    60  	if !redo.IsValidConsistentStorage(storageType) {
    61  		return nil, errors.ErrConsistentStorage.GenWithStackByArgs(storageType)
    62  	}
    63  	if redo.IsBlackholeStorage(storageType) {
    64  		return newBlackHoleReader(), nil
    65  	}
    66  	return newLogReader(ctx, cfg)
    67  }
    68  
    69  // LogReaderConfig is the config for LogReader
    70  type LogReaderConfig struct {
    71  	// Dir is the folder contains the redo logs need to apply when OP environment or
    72  	// the folder used to download redo logs to if using external storage, such as s3
    73  	// and gcs.
    74  	Dir string
    75  
    76  	// URI should be like "s3://logbucket/test-changefeed?endpoint=http://$S3_ENDPOINT/"
    77  	URI                url.URL
    78  	UseExternalStorage bool
    79  
    80  	// WorkerNums is the num of workers used to sort the log file to sorted file,
    81  	// will load the file to memory first then write the sorted file to disk
    82  	// the memory used is WorkerNums * defaultMaxLogSize (64 * megabyte) total
    83  	WorkerNums int
    84  }
    85  
    86  // LogReader implement RedoLogReader interface
    87  type LogReader struct {
    88  	cfg   *LogReaderConfig
    89  	meta  *common.LogMeta
    90  	rowCh chan *model.RowChangedEventInRedoLog
    91  	ddlCh chan *model.DDLEvent
    92  }
    93  
    94  // newLogReader creates a LogReader instance.
    95  // Need the client to guarantee only one LogReader per changefeed
    96  // currently support rewind operation by ResetReader api
    97  // if s3 will download logs first, if OP environment need fetch the redo logs to local dir first
    98  func newLogReader(ctx context.Context, cfg *LogReaderConfig) (*LogReader, error) {
    99  	if cfg == nil {
   100  		err := errors.New("LogReaderConfig can not be nil")
   101  		return nil, errors.WrapError(errors.ErrRedoConfigInvalid, err)
   102  	}
   103  	if cfg.WorkerNums == 0 {
   104  		cfg.WorkerNums = defaultWorkerNum
   105  	}
   106  
   107  	logReader := &LogReader{
   108  		cfg:   cfg,
   109  		rowCh: make(chan *model.RowChangedEventInRedoLog, defaultReaderChanSize),
   110  		ddlCh: make(chan *model.DDLEvent, defaultReaderChanSize),
   111  	}
   112  	// remove logs in local dir first, if have logs left belongs to previous changefeed with the same name may have error when apply logs
   113  	if err := os.RemoveAll(cfg.Dir); err != nil {
   114  		return nil, errors.WrapError(errors.ErrRedoFileOp, err)
   115  	}
   116  	if err := logReader.initMeta(ctx); err != nil {
   117  		return nil, err
   118  	}
   119  	return logReader, nil
   120  }
   121  
   122  // Run implements the `RedoLogReader` interface.
   123  func (l *LogReader) Run(ctx context.Context) error {
   124  	select {
   125  	case <-ctx.Done():
   126  		return errors.Trace(ctx.Err())
   127  	default:
   128  	}
   129  
   130  	if l.meta == nil {
   131  		return errors.Trace(errors.ErrRedoMetaFileNotFound.GenWithStackByArgs(l.cfg.Dir))
   132  	}
   133  
   134  	eg, egCtx := errgroup.WithContext(ctx)
   135  	eg.Go(func() error {
   136  		return l.runRowReader(egCtx)
   137  	})
   138  	eg.Go(func() error {
   139  		return l.runDDLReader(egCtx)
   140  	})
   141  	return eg.Wait()
   142  }
   143  
   144  func (l *LogReader) runRowReader(egCtx context.Context) error {
   145  	defer close(l.rowCh)
   146  	rowCfg := &readerConfig{
   147  		startTs:            l.meta.CheckpointTs,
   148  		endTs:              l.meta.ResolvedTs,
   149  		dir:                l.cfg.Dir,
   150  		fileType:           redo.RedoRowLogFileType,
   151  		uri:                l.cfg.URI,
   152  		useExternalStorage: l.cfg.UseExternalStorage,
   153  		workerNums:         l.cfg.WorkerNums,
   154  	}
   155  	return l.runReader(egCtx, rowCfg)
   156  }
   157  
   158  func (l *LogReader) runDDLReader(egCtx context.Context) error {
   159  	defer close(l.ddlCh)
   160  	ddlCfg := &readerConfig{
   161  		startTs:            l.meta.CheckpointTs - 1,
   162  		endTs:              l.meta.ResolvedTs,
   163  		dir:                l.cfg.Dir,
   164  		fileType:           redo.RedoDDLLogFileType,
   165  		uri:                l.cfg.URI,
   166  		useExternalStorage: l.cfg.UseExternalStorage,
   167  		workerNums:         l.cfg.WorkerNums,
   168  	}
   169  	return l.runReader(egCtx, ddlCfg)
   170  }
   171  
   172  func (l *LogReader) runReader(egCtx context.Context, cfg *readerConfig) error {
   173  	fileReaders, err := newReaders(egCtx, cfg)
   174  	if err != nil {
   175  		return errors.Trace(err)
   176  	}
   177  	defer func() {
   178  		var errs error
   179  		for _, r := range fileReaders {
   180  			errs = multierr.Append(errs, r.Close())
   181  		}
   182  		if errs != nil {
   183  			log.Error("close row reader failed", zap.Error(errs))
   184  		}
   185  	}()
   186  
   187  	// init heap
   188  	redoLogHeap, err := newLogHeap(fileReaders)
   189  	if err != nil {
   190  		return errors.Trace(err)
   191  	}
   192  
   193  	for redoLogHeap.Len() != 0 {
   194  		item := heap.Pop(&redoLogHeap).(*logWithIdx)
   195  
   196  		switch cfg.fileType {
   197  		case redo.RedoRowLogFileType:
   198  			row := item.data.RedoRow.Row
   199  			// By design only data (startTs,endTs] is needed,
   200  			// so filter out data may beyond the boundary.
   201  			if row != nil && row.CommitTs > cfg.startTs && row.CommitTs <= cfg.endTs {
   202  				select {
   203  				case <-egCtx.Done():
   204  					return errors.Trace(egCtx.Err())
   205  				case l.rowCh <- row:
   206  				}
   207  			}
   208  		case redo.RedoDDLLogFileType:
   209  			ddl := item.data.RedoDDL.DDL
   210  			if ddl != nil && ddl.CommitTs > cfg.startTs && ddl.CommitTs <= cfg.endTs {
   211  				select {
   212  				case <-egCtx.Done():
   213  					return errors.Trace(egCtx.Err())
   214  				case l.ddlCh <- ddl:
   215  				}
   216  			}
   217  		}
   218  
   219  		// read next and push again
   220  		rl, err := fileReaders[item.idx].Read()
   221  		if err != nil {
   222  			if err != io.EOF {
   223  				return errors.Trace(err)
   224  			}
   225  			continue
   226  		}
   227  		ld := &logWithIdx{
   228  			data: rl,
   229  			idx:  item.idx,
   230  		}
   231  		heap.Push(&redoLogHeap, ld)
   232  	}
   233  	return nil
   234  }
   235  
   236  // ReadNextRow implement the `RedoLogReader` interface.
   237  func (l *LogReader) ReadNextRow(ctx context.Context) (*model.RowChangedEvent, error) {
   238  	select {
   239  	case <-ctx.Done():
   240  		return nil, errors.Trace(ctx.Err())
   241  	case rowInRedoLog := <-l.rowCh:
   242  		if rowInRedoLog != nil {
   243  			return rowInRedoLog.ToRowChangedEvent(), nil
   244  		}
   245  		return nil, nil
   246  	}
   247  }
   248  
   249  // ReadNextDDL implement the `RedoLogReader` interface.
   250  func (l *LogReader) ReadNextDDL(ctx context.Context) (*model.DDLEvent, error) {
   251  	select {
   252  	case <-ctx.Done():
   253  		return nil, errors.Trace(ctx.Err())
   254  	case ddl := <-l.ddlCh:
   255  		return ddl, nil
   256  	}
   257  }
   258  
   259  func (l *LogReader) initMeta(ctx context.Context) error {
   260  	select {
   261  	case <-ctx.Done():
   262  		return errors.Trace(ctx.Err())
   263  	default:
   264  	}
   265  	extStorage, err := redo.InitExternalStorage(ctx, l.cfg.URI)
   266  	if err != nil {
   267  		return err
   268  	}
   269  	metas := make([]*common.LogMeta, 0, 64)
   270  	err = extStorage.WalkDir(ctx, nil, func(path string, size int64) error {
   271  		if !strings.HasSuffix(path, redo.MetaEXT) {
   272  			return nil
   273  		}
   274  
   275  		data, err := extStorage.ReadFile(ctx, path)
   276  		if err != nil && !util.IsNotExistInExtStorage(err) {
   277  			return err
   278  		}
   279  		if len(data) != 0 {
   280  			var meta common.LogMeta
   281  			_, err = meta.UnmarshalMsg(data)
   282  			if err != nil {
   283  				return err
   284  			}
   285  			metas = append(metas, &meta)
   286  		}
   287  		return nil
   288  	})
   289  	if err != nil {
   290  		return errors.WrapError(errors.ErrRedoMetaInitialize,
   291  			errors.Annotate(err, "read meta file fail"))
   292  	}
   293  	if len(metas) == 0 {
   294  		return errors.ErrRedoMetaFileNotFound.GenWithStackByArgs(l.cfg.Dir)
   295  	}
   296  
   297  	var checkpointTs, resolvedTs uint64
   298  	common.ParseMeta(metas, &checkpointTs, &resolvedTs)
   299  	if resolvedTs < checkpointTs {
   300  		log.Panic("in all meta files, resolvedTs is less than checkpointTs",
   301  			zap.Uint64("resolvedTs", resolvedTs),
   302  			zap.Uint64("checkpointTs", checkpointTs))
   303  	}
   304  	l.meta = &common.LogMeta{CheckpointTs: checkpointTs, ResolvedTs: resolvedTs}
   305  	return nil
   306  }
   307  
   308  // ReadMeta implement ReadMeta interface
   309  func (l *LogReader) ReadMeta(ctx context.Context) (checkpointTs, resolvedTs uint64, err error) {
   310  	if l.meta == nil {
   311  		return 0, 0, errors.Trace(errors.ErrRedoMetaFileNotFound.GenWithStackByArgs(l.cfg.Dir))
   312  	}
   313  	return l.meta.CheckpointTs, l.meta.ResolvedTs, nil
   314  }
   315  
   316  type logWithIdx struct {
   317  	idx  int
   318  	data *model.RedoLog
   319  }
   320  
   321  type logHeap []*logWithIdx
   322  
   323  func newLogHeap(fileReaders []fileReader) (logHeap, error) {
   324  	h := logHeap{}
   325  	for i := 0; i < len(fileReaders); i++ {
   326  		rl, err := fileReaders[i].Read()
   327  		if err != nil {
   328  			if err != io.EOF {
   329  				return nil, err
   330  			}
   331  			continue
   332  		}
   333  
   334  		ld := &logWithIdx{
   335  			data: rl,
   336  			idx:  i,
   337  		}
   338  		h = append(h, ld)
   339  	}
   340  	heap.Init(&h)
   341  	return h, nil
   342  }
   343  
   344  func (h logHeap) Len() int {
   345  	return len(h)
   346  }
   347  
   348  func (h logHeap) Less(i, j int) bool {
   349  	// we separate ddl and dml, so we only need to compare dml with dml, and ddl with ddl.
   350  	if h[i].data.Type == model.RedoLogTypeDDL {
   351  		if h[i].data.RedoDDL.DDL == nil {
   352  			return true
   353  		}
   354  		if h[j].data.RedoDDL.DDL == nil {
   355  			return false
   356  		}
   357  		return h[i].data.RedoDDL.DDL.CommitTs < h[j].data.RedoDDL.DDL.CommitTs
   358  	}
   359  
   360  	if h[i].data.RedoRow.Row == nil {
   361  		return true
   362  	}
   363  	if h[j].data.RedoRow.Row == nil {
   364  		return false
   365  	}
   366  
   367  	if h[i].data.RedoRow.Row.CommitTs == h[j].data.RedoRow.Row.CommitTs {
   368  		if h[i].data.RedoRow.Row.StartTs != h[j].data.RedoRow.Row.StartTs {
   369  			return h[i].data.RedoRow.Row.StartTs < h[j].data.RedoRow.Row.StartTs
   370  		}
   371  		// in the same txn, we need to sort by delete/update/insert order
   372  		if h[i].data.RedoRow.Row.ToRowChangedEvent().IsDelete() {
   373  			return true
   374  		} else if h[i].data.RedoRow.Row.ToRowChangedEvent().IsUpdate() {
   375  			return !h[j].data.RedoRow.Row.ToRowChangedEvent().IsDelete()
   376  		}
   377  		return false
   378  	}
   379  
   380  	return h[i].data.RedoRow.Row.CommitTs < h[j].data.RedoRow.Row.CommitTs
   381  }
   382  
   383  func (h logHeap) Swap(i, j int) {
   384  	h[i], h[j] = h[j], h[i]
   385  }
   386  
   387  func (h *logHeap) Push(x interface{}) {
   388  	*h = append(*h, x.(*logWithIdx))
   389  }
   390  
   391  func (h *logHeap) Pop() interface{} {
   392  	old := *h
   393  	n := len(old)
   394  	x := old[n-1]
   395  	*h = old[0 : n-1]
   396  	return x
   397  }