github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/redo/writer/memory/encoding_worker.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  	"bytes"
    18  	"context"
    19  	"encoding/binary"
    20  	"sync"
    21  	"sync/atomic"
    22  
    23  	"github.com/pingcap/log"
    24  	"github.com/pingcap/tiflow/cdc/model"
    25  	"github.com/pingcap/tiflow/cdc/model/codec"
    26  	"github.com/pingcap/tiflow/cdc/redo/writer"
    27  	"github.com/pingcap/tiflow/pkg/errors"
    28  	"github.com/pingcap/tiflow/pkg/redo"
    29  	"go.uber.org/zap"
    30  	"golang.org/x/sync/errgroup"
    31  )
    32  
    33  var (
    34  	eventPool = sync.Pool{
    35  		New: func() interface{} {
    36  			return &polymorphicRedoEvent{}
    37  		},
    38  	}
    39  	dataPool = sync.Pool{
    40  		New: func() interface{} {
    41  			return new(bytes.Buffer)
    42  		},
    43  	}
    44  )
    45  
    46  type polymorphicRedoEvent struct {
    47  	// event is the redo event (model.RowChangedEvent or model.DDLEvent) to be encoded.
    48  	event writer.RedoEvent
    49  	// data is the encoded redo event.
    50  	data     *bytes.Buffer
    51  	commitTs model.Ts
    52  
    53  	flushCallback func()
    54  }
    55  
    56  func newFlushPolymorphicRedoLog(fn func()) *polymorphicRedoEvent {
    57  	return &polymorphicRedoEvent{
    58  		flushCallback: fn,
    59  	}
    60  }
    61  
    62  func (e *polymorphicRedoEvent) reset() {
    63  	e.event = nil
    64  	dataPool.Put(e.data)
    65  	e.data = nil
    66  	e.commitTs = 0
    67  }
    68  
    69  // encoding format: lenField(8 bytes) + rawData + padding bytes(force 8 bytes alignment)
    70  func (e *polymorphicRedoEvent) encode() (err error) {
    71  	redoLog := e.event.ToRedoLog()
    72  	e.commitTs = redoLog.GetCommitTs()
    73  
    74  	rawData, err := codec.MarshalRedoLog(redoLog, nil)
    75  	if err != nil {
    76  		return err
    77  	}
    78  	uint64buf := make([]byte, 8)
    79  	lenField, padBytes := writer.EncodeFrameSize(len(rawData))
    80  	binary.LittleEndian.PutUint64(uint64buf, lenField)
    81  
    82  	e.data = dataPool.Get().(*bytes.Buffer)
    83  	e.data.Reset()
    84  	_, err = e.data.Write(uint64buf)
    85  	if err != nil {
    86  		return err
    87  	}
    88  	_, err = e.data.Write(rawData)
    89  	if err != nil {
    90  		return err
    91  	}
    92  	if padBytes != 0 {
    93  		_, err = e.data.Write(make([]byte, padBytes))
    94  	}
    95  
    96  	e.event = nil
    97  	return err
    98  }
    99  
   100  type encodingWorkerGroup struct {
   101  	changefeed model.ChangeFeedID
   102  	outputCh   chan *polymorphicRedoEvent
   103  	inputChs   []chan *polymorphicRedoEvent
   104  	workerNum  int
   105  	nextWorker atomic.Uint64
   106  
   107  	closed chan error
   108  }
   109  
   110  func newEncodingWorkerGroup(cfg *writer.LogWriterConfig) *encodingWorkerGroup {
   111  	workerNum := cfg.EncodingWorkerNum
   112  	if workerNum <= 0 {
   113  		workerNum = redo.DefaultEncodingWorkerNum
   114  	}
   115  	inputChs := make([]chan *polymorphicRedoEvent, workerNum)
   116  	for i := 0; i < workerNum; i++ {
   117  		inputChs[i] = make(chan *polymorphicRedoEvent, redo.DefaultEncodingInputChanSize)
   118  	}
   119  	return &encodingWorkerGroup{
   120  		changefeed: cfg.ChangeFeedID,
   121  		inputChs:   inputChs,
   122  		outputCh:   make(chan *polymorphicRedoEvent, redo.DefaultEncodingOutputChanSize),
   123  		workerNum:  workerNum,
   124  		closed:     make(chan error, 1),
   125  	}
   126  }
   127  
   128  func (e *encodingWorkerGroup) Run(ctx context.Context) (err error) {
   129  	defer func() {
   130  		log.Warn("redo encoding workers closed",
   131  			zap.String("namespace", e.changefeed.Namespace),
   132  			zap.String("changefeed", e.changefeed.ID),
   133  			zap.Error(err))
   134  		if err != nil && errors.Cause(err) != context.Canceled {
   135  			e.closed <- err
   136  		}
   137  		close(e.closed)
   138  	}()
   139  	eg, egCtx := errgroup.WithContext(ctx)
   140  	for i := 0; i < e.workerNum; i++ {
   141  		idx := i
   142  		eg.Go(func() error {
   143  			return e.runWorker(egCtx, idx)
   144  		})
   145  	}
   146  	log.Info("redo log encoding workers started",
   147  		zap.String("namespace", e.changefeed.Namespace),
   148  		zap.String("changefeed", e.changefeed.ID),
   149  		zap.Int("workerNum", e.workerNum))
   150  	return eg.Wait()
   151  }
   152  
   153  func (e *encodingWorkerGroup) AddEvent(ctx context.Context, event writer.RedoEvent) error {
   154  	redoEvent := eventPool.Get().(*polymorphicRedoEvent)
   155  	redoEvent.event = event
   156  	idx := e.nextWorker.Add(1) % uint64(e.workerNum)
   157  	return e.input(ctx, idx, redoEvent)
   158  }
   159  
   160  func (e *encodingWorkerGroup) runWorker(egCtx context.Context, idx int) error {
   161  	for {
   162  		select {
   163  		case <-egCtx.Done():
   164  			return errors.Trace(egCtx.Err())
   165  		case event := <-e.inputChs[idx]:
   166  			if event.event != nil {
   167  				if err := event.encode(); err != nil {
   168  					return errors.Trace(err)
   169  				}
   170  				if err := e.output(egCtx, event); err != nil {
   171  					return errors.Trace(err)
   172  				}
   173  			}
   174  			if event.flushCallback != nil {
   175  				event.flushCallback()
   176  			}
   177  		}
   178  	}
   179  }
   180  
   181  func (e *encodingWorkerGroup) input(
   182  	ctx context.Context, idx uint64, event *polymorphicRedoEvent,
   183  ) error {
   184  	select {
   185  	case <-ctx.Done():
   186  		return ctx.Err()
   187  	case err := <-e.closed:
   188  		return errors.WrapError(errors.ErrRedoWriterStopped, err, "encoding worker is closed")
   189  	case e.inputChs[idx] <- event:
   190  		return nil
   191  	}
   192  }
   193  
   194  func (e *encodingWorkerGroup) output(
   195  	ctx context.Context, event *polymorphicRedoEvent,
   196  ) error {
   197  	select {
   198  	case <-ctx.Done():
   199  		return ctx.Err()
   200  	case err := <-e.closed:
   201  		return errors.WrapError(errors.ErrRedoWriterStopped, err, "encoding worker is closed")
   202  	case e.outputCh <- event:
   203  		return nil
   204  	}
   205  }
   206  
   207  func (e *encodingWorkerGroup) FlushAll(ctx context.Context) error {
   208  	if err := e.broadcastAndWaitEncoding(ctx); err != nil {
   209  		return err
   210  	}
   211  
   212  	// notify file worker to flush
   213  	flushCh := make(chan struct{})
   214  	flushEvent := newFlushPolymorphicRedoLog(func() {
   215  		close(flushCh)
   216  	})
   217  	if err := e.output(ctx, flushEvent); err != nil {
   218  		return err
   219  	}
   220  
   221  	// wait all file flushed to external storage
   222  	select {
   223  	case <-ctx.Done():
   224  		return ctx.Err()
   225  	case err := <-e.closed:
   226  		return errors.WrapError(errors.ErrRedoWriterStopped, err, "encoding worker is closed")
   227  	case <-flushCh:
   228  	}
   229  	return nil
   230  }
   231  
   232  func (e *encodingWorkerGroup) broadcastAndWaitEncoding(ctx context.Context) error {
   233  	flushChs := make([]chan struct{}, e.workerNum)
   234  	for i := 0; i < e.workerNum; i++ {
   235  		ch := make(chan struct{})
   236  		flushEvent := newFlushPolymorphicRedoLog(func() {
   237  			close(ch)
   238  		})
   239  		if err := e.input(ctx, uint64(i), flushEvent); err != nil {
   240  			return err
   241  		}
   242  		flushChs[i] = ch
   243  	}
   244  
   245  	for _, ch := range flushChs {
   246  		select {
   247  		case <-ctx.Done():
   248  			return ctx.Err()
   249  		case err := <-e.closed:
   250  			return errors.WrapError(errors.ErrRedoWriterStopped, err, "encoding worker is closed")
   251  		case <-ch:
   252  		}
   253  	}
   254  	return nil
   255  }