github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/redo/writer/memory/mem_log_writer.go (about)

     1  //  Copyright 2023 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 memory
    15  
    16  import (
    17  	"context"
    18  
    19  	"github.com/pingcap/log"
    20  	"github.com/pingcap/tiflow/cdc/redo/writer"
    21  	"github.com/pingcap/tiflow/pkg/errors"
    22  	"github.com/pingcap/tiflow/pkg/redo"
    23  	"go.uber.org/zap"
    24  	"golang.org/x/sync/errgroup"
    25  )
    26  
    27  var _ writer.RedoLogWriter = (*memoryLogWriter)(nil)
    28  
    29  type memoryLogWriter struct {
    30  	cfg           *writer.LogWriterConfig
    31  	encodeWorkers *encodingWorkerGroup
    32  	fileWorkers   *fileWorkerGroup
    33  
    34  	eg     *errgroup.Group
    35  	cancel context.CancelFunc
    36  }
    37  
    38  // NewLogWriter creates a new memoryLogWriter.
    39  func NewLogWriter(
    40  	ctx context.Context, cfg *writer.LogWriterConfig, opts ...writer.Option,
    41  ) (*memoryLogWriter, error) {
    42  	if cfg == nil {
    43  		return nil, errors.WrapError(errors.ErrRedoConfigInvalid,
    44  			errors.New("invalid LogWriterConfig"))
    45  	}
    46  
    47  	// "nfs" and "local" scheme are converted to "file" scheme
    48  	if !cfg.UseExternalStorage {
    49  		redo.FixLocalScheme(cfg.URI)
    50  		cfg.UseExternalStorage = redo.IsExternalStorage(cfg.URI.Scheme)
    51  	}
    52  
    53  	extStorage, err := redo.InitExternalStorage(ctx, *cfg.URI)
    54  	if err != nil {
    55  		return nil, err
    56  	}
    57  
    58  	eg, ctx := errgroup.WithContext(ctx)
    59  	lwCtx, lwCancel := context.WithCancel(ctx)
    60  	lw := &memoryLogWriter{
    61  		cfg:           cfg,
    62  		encodeWorkers: newEncodingWorkerGroup(cfg),
    63  		fileWorkers:   newFileWorkerGroup(cfg, cfg.FlushWorkerNum, extStorage, opts...),
    64  		eg:            eg,
    65  		cancel:        lwCancel,
    66  	}
    67  
    68  	eg.Go(func() error {
    69  		return lw.encodeWorkers.Run(lwCtx)
    70  	})
    71  	eg.Go(func() error {
    72  		return lw.fileWorkers.Run(lwCtx, lw.encodeWorkers.outputCh)
    73  	})
    74  	return lw, nil
    75  }
    76  
    77  // WriteEvents implements RedoLogWriter.WriteEvents
    78  func (l *memoryLogWriter) WriteEvents(ctx context.Context, events ...writer.RedoEvent) error {
    79  	for _, event := range events {
    80  		if event == nil {
    81  			log.Warn("writing nil event to redo log, ignore this",
    82  				zap.String("namespace", l.cfg.ChangeFeedID.Namespace),
    83  				zap.String("changefeed", l.cfg.ChangeFeedID.ID),
    84  				zap.String("capture", l.cfg.CaptureID))
    85  			continue
    86  		}
    87  		if err := l.encodeWorkers.AddEvent(ctx, event); err != nil {
    88  			return err
    89  		}
    90  	}
    91  	return nil
    92  }
    93  
    94  // FlushLog implement FlushLog api
    95  func (l *memoryLogWriter) FlushLog(ctx context.Context) error {
    96  	return l.encodeWorkers.FlushAll(ctx)
    97  }
    98  
    99  // Close implements RedoLogWriter.Close
   100  func (l *memoryLogWriter) Close() error {
   101  	if l.cancel != nil {
   102  		l.cancel()
   103  	} else {
   104  		log.Panic("redo writer close without init")
   105  	}
   106  	return l.eg.Wait()
   107  }