github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/manager.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 sink
    15  
    16  import (
    17  	"context"
    18  	"math"
    19  	"sort"
    20  	"sync"
    21  	"sync/atomic"
    22  	"time"
    23  
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/log"
    26  	"github.com/pingcap/ticdc/cdc/model"
    27  	"github.com/pingcap/ticdc/pkg/util"
    28  	"go.uber.org/zap"
    29  )
    30  
    31  const (
    32  	defaultMetricInterval = time.Second * 15
    33  )
    34  
    35  // Manager manages table sinks, maintains the relationship between table sinks and backendSink
    36  type Manager struct {
    37  	backendSink  Sink
    38  	checkpointTs model.Ts
    39  	tableSinks   map[model.TableID]*tableSink
    40  	tableSinksMu sync.Mutex
    41  
    42  	flushMu sync.Mutex
    43  
    44  	drawbackChan chan drawbackMsg
    45  }
    46  
    47  // NewManager creates a new Sink manager
    48  func NewManager(ctx context.Context, backendSink Sink, errCh chan error, checkpointTs model.Ts) *Manager {
    49  	drawbackChan := make(chan drawbackMsg, 16)
    50  	return &Manager{
    51  		backendSink:  newBufferSink(ctx, backendSink, errCh, checkpointTs, drawbackChan),
    52  		checkpointTs: checkpointTs,
    53  		tableSinks:   make(map[model.TableID]*tableSink),
    54  		drawbackChan: drawbackChan,
    55  	}
    56  }
    57  
    58  // CreateTableSink creates a table sink
    59  func (m *Manager) CreateTableSink(tableID model.TableID, checkpointTs model.Ts) Sink {
    60  	m.tableSinksMu.Lock()
    61  	defer m.tableSinksMu.Unlock()
    62  	if _, exist := m.tableSinks[tableID]; exist {
    63  		log.Panic("the table sink already exists", zap.Uint64("tableID", uint64(tableID)))
    64  	}
    65  	sink := &tableSink{
    66  		tableID:   tableID,
    67  		manager:   m,
    68  		buffer:    make([]*model.RowChangedEvent, 0, 128),
    69  		emittedTs: checkpointTs,
    70  	}
    71  	m.tableSinks[tableID] = sink
    72  	return sink
    73  }
    74  
    75  // Close closes the Sink manager and backend Sink, this method can be reentrantly called
    76  func (m *Manager) Close(ctx context.Context) error {
    77  	return m.backendSink.Close(ctx)
    78  }
    79  
    80  func (m *Manager) getMinEmittedTs() model.Ts {
    81  	m.tableSinksMu.Lock()
    82  	defer m.tableSinksMu.Unlock()
    83  	if len(m.tableSinks) == 0 {
    84  		return m.getCheckpointTs()
    85  	}
    86  	minTs := model.Ts(math.MaxUint64)
    87  	for _, tableSink := range m.tableSinks {
    88  		emittedTs := tableSink.getEmittedTs()
    89  		if minTs > emittedTs {
    90  			minTs = emittedTs
    91  		}
    92  	}
    93  	return minTs
    94  }
    95  
    96  func (m *Manager) flushBackendSink(ctx context.Context) (model.Ts, error) {
    97  	m.flushMu.Lock()
    98  	defer m.flushMu.Unlock()
    99  	minEmittedTs := m.getMinEmittedTs()
   100  	checkpointTs, err := m.backendSink.FlushRowChangedEvents(ctx, minEmittedTs)
   101  	if err != nil {
   102  		return m.getCheckpointTs(), errors.Trace(err)
   103  	}
   104  	atomic.StoreUint64(&m.checkpointTs, checkpointTs)
   105  	return checkpointTs, nil
   106  }
   107  
   108  func (m *Manager) destroyTableSink(ctx context.Context, tableID model.TableID) error {
   109  	m.tableSinksMu.Lock()
   110  	delete(m.tableSinks, tableID)
   111  	m.tableSinksMu.Unlock()
   112  	callback := make(chan struct{})
   113  	select {
   114  	case <-ctx.Done():
   115  		return ctx.Err()
   116  	case m.drawbackChan <- drawbackMsg{tableID: tableID, callback: callback}:
   117  	}
   118  	select {
   119  	case <-ctx.Done():
   120  		return ctx.Err()
   121  	case <-callback:
   122  	}
   123  	return m.backendSink.Barrier(ctx)
   124  }
   125  
   126  func (m *Manager) getCheckpointTs() uint64 {
   127  	return atomic.LoadUint64(&m.checkpointTs)
   128  }
   129  
   130  type tableSink struct {
   131  	tableID model.TableID
   132  	manager *Manager
   133  	buffer  []*model.RowChangedEvent
   134  	// emittedTs means all of events which of commitTs less than or equal to emittedTs is sent to backendSink
   135  	emittedTs model.Ts
   136  }
   137  
   138  func (t *tableSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error {
   139  	// do nothing
   140  	return nil
   141  }
   142  
   143  func (t *tableSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error {
   144  	t.buffer = append(t.buffer, rows...)
   145  	return nil
   146  }
   147  
   148  func (t *tableSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error {
   149  	// the table sink doesn't receive the DDL event
   150  	return nil
   151  }
   152  
   153  func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) {
   154  	i := sort.Search(len(t.buffer), func(i int) bool {
   155  		return t.buffer[i].CommitTs > resolvedTs
   156  	})
   157  	if i == 0 {
   158  		atomic.StoreUint64(&t.emittedTs, resolvedTs)
   159  		return t.manager.flushBackendSink(ctx)
   160  	}
   161  	resolvedRows := t.buffer[:i]
   162  	t.buffer = append(make([]*model.RowChangedEvent, 0, len(t.buffer[i:])), t.buffer[i:]...)
   163  
   164  	err := t.manager.backendSink.EmitRowChangedEvents(ctx, resolvedRows...)
   165  	if err != nil {
   166  		return t.manager.getCheckpointTs(), errors.Trace(err)
   167  	}
   168  	atomic.StoreUint64(&t.emittedTs, resolvedTs)
   169  	return t.manager.flushBackendSink(ctx)
   170  }
   171  
   172  func (t *tableSink) getEmittedTs() uint64 {
   173  	return atomic.LoadUint64(&t.emittedTs)
   174  }
   175  
   176  func (t *tableSink) EmitCheckpointTs(ctx context.Context, ts uint64) error {
   177  	// the table sink doesn't receive the checkpoint event
   178  	return nil
   179  }
   180  
   181  // Note once the Close is called, no more events can be written to this table sink
   182  func (t *tableSink) Close(ctx context.Context) error {
   183  	return t.manager.destroyTableSink(ctx, t.tableID)
   184  }
   185  
   186  // Barrier is not used in table sink
   187  func (t *tableSink) Barrier(ctx context.Context) error {
   188  	return nil
   189  }
   190  
   191  type drawbackMsg struct {
   192  	tableID  model.TableID
   193  	callback chan struct{}
   194  }
   195  
   196  type bufferSink struct {
   197  	Sink
   198  	checkpointTs uint64
   199  	buffer       map[model.TableID][]*model.RowChangedEvent
   200  	bufferMu     sync.Mutex
   201  	flushTsChan  chan uint64
   202  	drawbackChan chan drawbackMsg
   203  }
   204  
   205  func newBufferSink(
   206  	ctx context.Context,
   207  	backendSink Sink,
   208  	errCh chan error,
   209  	checkpointTs model.Ts,
   210  	drawbackChan chan drawbackMsg,
   211  ) Sink {
   212  	sink := &bufferSink{
   213  		Sink: backendSink,
   214  		// buffer shares the same flow control with table sink
   215  		buffer:       make(map[model.TableID][]*model.RowChangedEvent),
   216  		checkpointTs: checkpointTs,
   217  		flushTsChan:  make(chan uint64, 128),
   218  		drawbackChan: drawbackChan,
   219  	}
   220  	go sink.run(ctx, errCh)
   221  	return sink
   222  }
   223  
   224  func (b *bufferSink) run(ctx context.Context, errCh chan error) {
   225  	changefeedID := util.ChangefeedIDFromCtx(ctx)
   226  	advertiseAddr := util.CaptureAddrFromCtx(ctx)
   227  	metricFlushDuration := flushRowChangedDuration.WithLabelValues(advertiseAddr, changefeedID, "Flush")
   228  	metricEmitRowDuration := flushRowChangedDuration.WithLabelValues(advertiseAddr, changefeedID, "EmitRow")
   229  	metricBufferSize := bufferChanSizeGauge.WithLabelValues(advertiseAddr, changefeedID)
   230  	for {
   231  		select {
   232  		case <-ctx.Done():
   233  			err := ctx.Err()
   234  			if err != nil && errors.Cause(err) != context.Canceled {
   235  				errCh <- err
   236  			}
   237  			return
   238  		case drawback := <-b.drawbackChan:
   239  			b.bufferMu.Lock()
   240  			delete(b.buffer, drawback.tableID)
   241  			b.bufferMu.Unlock()
   242  			close(drawback.callback)
   243  		case resolvedTs := <-b.flushTsChan:
   244  			b.bufferMu.Lock()
   245  			// find all rows before resolvedTs and emit to backend sink
   246  			for tableID, rows := range b.buffer {
   247  				i := sort.Search(len(rows), func(i int) bool {
   248  					return rows[i].CommitTs > resolvedTs
   249  				})
   250  
   251  				start := time.Now()
   252  				err := b.Sink.EmitRowChangedEvents(ctx, rows[:i]...)
   253  				if err != nil {
   254  					b.bufferMu.Unlock()
   255  					if errors.Cause(err) != context.Canceled {
   256  						errCh <- err
   257  					}
   258  					return
   259  				}
   260  				dur := time.Since(start)
   261  				metricEmitRowDuration.Observe(dur.Seconds())
   262  
   263  				// put remaining rows back to buffer
   264  				// append to a new, fixed slice to avoid lazy GC
   265  				b.buffer[tableID] = append(make([]*model.RowChangedEvent, 0, len(rows[i:])), rows[i:]...)
   266  			}
   267  			b.bufferMu.Unlock()
   268  
   269  			start := time.Now()
   270  			checkpointTs, err := b.Sink.FlushRowChangedEvents(ctx, resolvedTs)
   271  			if err != nil {
   272  				if errors.Cause(err) != context.Canceled {
   273  					errCh <- err
   274  				}
   275  				return
   276  			}
   277  			atomic.StoreUint64(&b.checkpointTs, checkpointTs)
   278  
   279  			dur := time.Since(start)
   280  			metricFlushDuration.Observe(dur.Seconds())
   281  			if dur > 3*time.Second {
   282  				log.Warn("flush row changed events too slow",
   283  					zap.Duration("duration", dur), util.ZapFieldChangefeed(ctx))
   284  			}
   285  		case <-time.After(defaultMetricInterval):
   286  			metricBufferSize.Set(float64(len(b.buffer)))
   287  		}
   288  	}
   289  }
   290  
   291  func (b *bufferSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error {
   292  	select {
   293  	case <-ctx.Done():
   294  		return ctx.Err()
   295  	default:
   296  		if len(rows) == 0 {
   297  			return nil
   298  		}
   299  		tableID := rows[0].Table.TableID
   300  		b.bufferMu.Lock()
   301  		b.buffer[tableID] = append(b.buffer[tableID], rows...)
   302  		b.bufferMu.Unlock()
   303  	}
   304  	return nil
   305  }
   306  
   307  func (b *bufferSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) {
   308  	select {
   309  	case <-ctx.Done():
   310  		return atomic.LoadUint64(&b.checkpointTs), ctx.Err()
   311  	case b.flushTsChan <- resolvedTs:
   312  	}
   313  	return atomic.LoadUint64(&b.checkpointTs), nil
   314  }