github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/cdclog/file.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 cdclog
    15  
    16  import (
    17  	"context"
    18  	"net/url"
    19  	"os"
    20  	"path/filepath"
    21  
    22  	"github.com/pingcap/errors"
    23  	"github.com/pingcap/log"
    24  	parsemodel "github.com/pingcap/parser/model"
    25  	"github.com/pingcap/ticdc/cdc/model"
    26  	"github.com/pingcap/ticdc/cdc/sink/codec"
    27  	cerror "github.com/pingcap/ticdc/pkg/errors"
    28  	"github.com/pingcap/ticdc/pkg/quotes"
    29  	"github.com/uber-go/atomic"
    30  	"go.uber.org/zap"
    31  )
    32  
    33  const (
    34  	defaultDirMode  = 0o755
    35  	defaultFileMode = 0o644
    36  
    37  	defaultFileName = "cdclog"
    38  
    39  	maxRowFileSize = 10 << 20 // TODO update
    40  )
    41  
    42  type logPath struct {
    43  	root string
    44  	ddl  string
    45  	meta string
    46  }
    47  
    48  type tableStream struct {
    49  	dataCh  chan *model.RowChangedEvent
    50  	rowFile *os.File
    51  
    52  	encoder codec.EventBatchEncoder
    53  
    54  	tableID    int64
    55  	sendEvents *atomic.Int64
    56  	sendSize   *atomic.Int64
    57  }
    58  
    59  func newTableStream(tableID int64) logUnit {
    60  	return &tableStream{
    61  		tableID: tableID,
    62  		dataCh:  make(chan *model.RowChangedEvent, defaultBufferChanSize),
    63  
    64  		sendEvents: atomic.NewInt64(0),
    65  		sendSize:   atomic.NewInt64(0),
    66  	}
    67  }
    68  
    69  func (ts *tableStream) dataChan() chan *model.RowChangedEvent {
    70  	return ts.dataCh
    71  }
    72  
    73  func (ts *tableStream) TableID() int64 {
    74  	return ts.tableID
    75  }
    76  
    77  func (ts *tableStream) Events() *atomic.Int64 {
    78  	return ts.sendEvents
    79  }
    80  
    81  func (ts *tableStream) Size() *atomic.Int64 {
    82  	return ts.sendSize
    83  }
    84  
    85  func (ts *tableStream) isEmpty() bool {
    86  	return ts.sendEvents.Load() == 0
    87  }
    88  
    89  func (ts *tableStream) shouldFlush() bool {
    90  	// if sendSize > 5 MB or data chennal is full, flush it
    91  	return ts.sendSize.Load() > maxPartFlushSize || ts.sendEvents.Load() == defaultBufferChanSize
    92  }
    93  
    94  func (ts *tableStream) flush(ctx context.Context, sink *logSink) error {
    95  	var fileName string
    96  	flushedEvents := ts.sendEvents.Load()
    97  	flushedSize := ts.sendSize.Load()
    98  	if flushedEvents == 0 {
    99  		log.Info("[flushTableStreams] no events to flush")
   100  		return nil
   101  	}
   102  	firstCreated := false
   103  	if ts.encoder == nil {
   104  		// create encoder for each file
   105  		ts.encoder = sink.encoder()
   106  		firstCreated = true
   107  	}
   108  	for event := int64(0); event < flushedEvents; event++ {
   109  		row := <-ts.dataCh
   110  		if event == flushedEvents-1 {
   111  			// the last event
   112  			fileName = makeTableFileName(row.CommitTs)
   113  		}
   114  		_, err := ts.encoder.AppendRowChangedEvent(row)
   115  		if err != nil {
   116  			return err
   117  		}
   118  	}
   119  	rowDatas := ts.encoder.MixedBuild(firstCreated)
   120  	defer func() {
   121  		if ts.encoder != nil {
   122  			ts.encoder.Reset()
   123  		}
   124  	}()
   125  
   126  	log.Debug("[flushTableStreams] build cdc log data",
   127  		zap.Int64("table id", ts.tableID),
   128  		zap.Int64("flushed size", flushedSize),
   129  		zap.Int64("flushed event", flushedEvents),
   130  		zap.Int("encode size", len(rowDatas)),
   131  		zap.String("file name", fileName),
   132  	)
   133  
   134  	tableDir := filepath.Join(sink.root(), makeTableDirectoryName(ts.tableID))
   135  
   136  	if ts.rowFile == nil {
   137  		// create new file to append data
   138  		err := os.MkdirAll(tableDir, defaultDirMode)
   139  		if err != nil {
   140  			return err
   141  		}
   142  		file, err := os.OpenFile(filepath.Join(tableDir, defaultFileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode)
   143  		if err != nil {
   144  			return err
   145  		}
   146  		ts.rowFile = file
   147  	}
   148  
   149  	_, err := ts.rowFile.Write(rowDatas)
   150  	if err != nil {
   151  		return err
   152  	}
   153  
   154  	stat, err := ts.rowFile.Stat()
   155  	if err != nil {
   156  		return err
   157  	}
   158  
   159  	if stat.Size() > maxRowFileSize {
   160  		// rotate file
   161  		err := ts.rowFile.Close()
   162  		if err != nil {
   163  			return err
   164  		}
   165  		oldPath := filepath.Join(tableDir, defaultFileName)
   166  		newPath := filepath.Join(tableDir, fileName)
   167  		err = os.Rename(oldPath, newPath)
   168  		if err != nil {
   169  			return err
   170  		}
   171  		file, err := os.OpenFile(filepath.Join(tableDir, defaultFileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode)
   172  		if err != nil {
   173  			return err
   174  		}
   175  		ts.rowFile = file
   176  		ts.encoder = nil
   177  	}
   178  
   179  	ts.sendEvents.Sub(flushedEvents)
   180  	ts.sendSize.Sub(flushedSize)
   181  	return nil
   182  }
   183  
   184  type fileSink struct {
   185  	*logSink
   186  
   187  	logMeta *logMeta
   188  	logPath *logPath
   189  
   190  	ddlFile *os.File
   191  
   192  	ddlEncoder codec.EventBatchEncoder
   193  }
   194  
   195  func (f *fileSink) flushLogMeta() error {
   196  	data, err := f.logMeta.Marshal()
   197  	if err != nil {
   198  		return cerror.WrapError(cerror.ErrMarshalFailed, err)
   199  	}
   200  	// FIXME: if initialize succeed, O_WRONLY is enough, but sometimes it will failed
   201  	tmpFileName := f.logPath.meta + ".tmp"
   202  	tmpFile, err := os.OpenFile(tmpFileName, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, defaultFileMode)
   203  	if err != nil {
   204  		return cerror.WrapError(cerror.ErrFileSinkFileOp, err)
   205  	}
   206  	_, err = tmpFile.Write(data)
   207  	if err != nil {
   208  		return cerror.WrapError(cerror.ErrFileSinkFileOp, err)
   209  	}
   210  	err = os.Rename(tmpFileName, f.logPath.meta)
   211  	return cerror.WrapError(cerror.ErrFileSinkFileOp, err)
   212  }
   213  
   214  func (f *fileSink) createDDLFile(commitTs uint64) (*os.File, error) {
   215  	fileName := makeDDLFileName(commitTs)
   216  	file, err := os.OpenFile(filepath.Join(f.logPath.ddl, fileName), os.O_CREATE|os.O_WRONLY|os.O_APPEND, defaultFileMode)
   217  	if err != nil {
   218  		log.Error("[EmitDDLEvent] create ddl file failed", zap.Error(err))
   219  		return nil, cerror.WrapError(cerror.ErrFileSinkFileOp, err)
   220  	}
   221  	return file, nil
   222  }
   223  
   224  func (f *fileSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error {
   225  	return f.emitRowChangedEvents(ctx, newTableStream, rows...)
   226  }
   227  
   228  func (f *fileSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) {
   229  	log.Debug("[FlushRowChangedEvents] enter", zap.Uint64("ts", resolvedTs))
   230  	return f.flushRowChangedEvents(ctx, resolvedTs)
   231  }
   232  
   233  func (f *fileSink) EmitCheckpointTs(ctx context.Context, ts uint64) error {
   234  	log.Debug("[EmitCheckpointTs]", zap.Uint64("ts", ts))
   235  	f.logMeta.GlobalResolvedTS = ts
   236  	return f.flushLogMeta()
   237  }
   238  
   239  func (f *fileSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error {
   240  	switch ddl.Type {
   241  	case parsemodel.ActionCreateTable:
   242  		f.logMeta.Names[ddl.TableInfo.TableID] = quotes.QuoteSchema(ddl.TableInfo.Schema, ddl.TableInfo.Table)
   243  		err := f.flushLogMeta()
   244  		if err != nil {
   245  			return err
   246  		}
   247  	case parsemodel.ActionRenameTable:
   248  		delete(f.logMeta.Names, ddl.PreTableInfo.TableID)
   249  		f.logMeta.Names[ddl.TableInfo.TableID] = quotes.QuoteSchema(ddl.TableInfo.Schema, ddl.TableInfo.Table)
   250  		err := f.flushLogMeta()
   251  		if err != nil {
   252  			return err
   253  		}
   254  	}
   255  	firstCreated := false
   256  	if f.ddlEncoder == nil {
   257  		// create ddl encoder once for each ddl log file
   258  		f.ddlEncoder = f.encoder()
   259  		firstCreated = true
   260  	}
   261  	_, err := f.ddlEncoder.EncodeDDLEvent(ddl)
   262  	if err != nil {
   263  		return err
   264  	}
   265  	data := f.ddlEncoder.MixedBuild(firstCreated)
   266  
   267  	defer func() {
   268  		if f.ddlEncoder != nil {
   269  			f.ddlEncoder.Reset()
   270  		}
   271  	}()
   272  
   273  	if f.ddlFile == nil {
   274  		// create file stream
   275  		file, err := f.createDDLFile(ddl.CommitTs)
   276  		if err != nil {
   277  			return err
   278  		}
   279  		f.ddlFile = file
   280  	}
   281  
   282  	stat, err := f.ddlFile.Stat()
   283  	if err != nil {
   284  		return cerror.WrapError(cerror.ErrFileSinkFileOp, err)
   285  	}
   286  
   287  	log.Debug("[EmitDDLEvent] current file stats",
   288  		zap.String("name", stat.Name()),
   289  		zap.Int64("size", stat.Size()),
   290  		zap.Int("data size", len(data)),
   291  	)
   292  
   293  	if stat.Size() > maxDDLFlushSize {
   294  		// rotate file
   295  		err = f.ddlFile.Close()
   296  		if err != nil {
   297  			return cerror.WrapError(cerror.ErrFileSinkFileOp, err)
   298  		}
   299  		file, err := f.createDDLFile(ddl.CommitTs)
   300  		if err != nil {
   301  			return err
   302  		}
   303  		f.ddlFile = file
   304  		// reset ddl encoder for new file
   305  		f.ddlEncoder = nil
   306  	}
   307  
   308  	_, err = f.ddlFile.Write(data)
   309  	if err != nil {
   310  		return cerror.WrapError(cerror.ErrFileSinkFileOp, err)
   311  	}
   312  	return nil
   313  }
   314  
   315  func (f *fileSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error {
   316  	if tableInfo != nil {
   317  		for _, table := range tableInfo {
   318  			if table != nil {
   319  				name := makeTableDirectoryName(table.TableID)
   320  				err := os.MkdirAll(filepath.Join(f.logPath.root, name), defaultDirMode)
   321  				if err != nil {
   322  					return cerror.WrapError(cerror.ErrFileSinkCreateDir, err)
   323  				}
   324  			}
   325  		}
   326  		// update log meta to record the relationship about tableName and tableID
   327  		f.logMeta = makeLogMetaContent(tableInfo)
   328  		data, err := f.logMeta.Marshal()
   329  		if err != nil {
   330  			return cerror.WrapError(cerror.ErrMarshalFailed, err)
   331  		}
   332  		filePath := f.logPath.meta
   333  		if _, err := os.Stat(filePath); !os.IsNotExist(err) {
   334  			return cerror.WrapError(cerror.ErrFileSinkMetaAlreadyExists, err)
   335  		}
   336  		file, err := os.Create(filePath)
   337  		if err != nil {
   338  			return cerror.WrapError(cerror.ErrFileSinkCreateDir, err)
   339  		}
   340  		_, err = file.Write(data)
   341  		if err != nil {
   342  			return cerror.WrapError(cerror.ErrFileSinkFileOp, err)
   343  		}
   344  	}
   345  	return nil
   346  }
   347  
   348  func (f *fileSink) Close(ctx context.Context) error {
   349  	return nil
   350  }
   351  
   352  func (f *fileSink) Barrier(ctx context.Context) error {
   353  	// Barrier does nothing because FlushRowChangedEvents in file sink has flushed
   354  	// all buffered events forcedlly.
   355  	return nil
   356  }
   357  
   358  // NewLocalFileSink support log data to file.
   359  func NewLocalFileSink(ctx context.Context, sinkURI *url.URL, errCh chan error) (*fileSink, error) {
   360  	log.Info("[NewLocalFileSink]",
   361  		zap.String("host", sinkURI.Host),
   362  		zap.String("path", sinkURI.Path),
   363  	)
   364  	rootPath := sinkURI.Path + "/"
   365  	logPath := &logPath{
   366  		root: rootPath,
   367  		meta: rootPath + logMetaFile,
   368  		ddl:  rootPath + ddlEventsDir,
   369  	}
   370  	err := os.MkdirAll(logPath.ddl, defaultDirMode)
   371  	if err != nil {
   372  		log.Error("create ddl path failed",
   373  			zap.String("ddl path", logPath.ddl),
   374  			zap.Error(err))
   375  		return nil, cerror.WrapError(cerror.ErrFileSinkCreateDir, err)
   376  	}
   377  
   378  	f := &fileSink{
   379  		logMeta: newLogMeta(),
   380  		logPath: logPath,
   381  		logSink: newLogSink(logPath.root, nil),
   382  	}
   383  
   384  	// important! we should flush asynchronously in another goroutine
   385  	go func() {
   386  		if err := f.startFlush(ctx); err != nil && errors.Cause(err) != context.Canceled {
   387  			select {
   388  			case <-ctx.Done():
   389  				return
   390  			case errCh <- err:
   391  			default:
   392  				log.Error("error channel is full", zap.Error(err))
   393  			}
   394  		}
   395  	}()
   396  	return f, nil
   397  }