github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/puller/entry_sorter.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 puller
    15  
    16  import (
    17  	"context"
    18  	"sort"
    19  	"sync"
    20  	"sync/atomic"
    21  	"time"
    22  
    23  	"github.com/pingcap/errors"
    24  	"github.com/pingcap/log"
    25  	"github.com/pingcap/ticdc/cdc/model"
    26  	"github.com/pingcap/ticdc/pkg/notify"
    27  	"github.com/pingcap/ticdc/pkg/util"
    28  	"go.uber.org/zap"
    29  	"golang.org/x/sync/errgroup"
    30  )
    31  
    32  // EntrySorter accepts out-of-order raw kv entries and output sorted entries
    33  type EntrySorter struct {
    34  	unsorted        []*model.PolymorphicEvent
    35  	lock            sync.Mutex
    36  	resolvedTsGroup []uint64
    37  	closed          int32
    38  
    39  	outputCh         chan *model.PolymorphicEvent
    40  	resolvedNotifier *notify.Notifier
    41  }
    42  
    43  // NewEntrySorter creates a new EntrySorter
    44  func NewEntrySorter() *EntrySorter {
    45  	return &EntrySorter{
    46  		resolvedNotifier: new(notify.Notifier),
    47  		outputCh:         make(chan *model.PolymorphicEvent, 128000),
    48  	}
    49  }
    50  
    51  // Run runs EntrySorter
    52  func (es *EntrySorter) Run(ctx context.Context) error {
    53  	captureAddr := util.CaptureAddrFromCtx(ctx)
    54  	changefeedID := util.ChangefeedIDFromCtx(ctx)
    55  	_, tableName := util.TableIDFromCtx(ctx)
    56  	metricEntrySorterResolvedChanSizeGuage := entrySorterResolvedChanSizeGauge.WithLabelValues(captureAddr, changefeedID, tableName)
    57  	metricEntrySorterOutputChanSizeGauge := entrySorterOutputChanSizeGauge.WithLabelValues(captureAddr, changefeedID, tableName)
    58  	metricEntryUnsortedSizeGauge := entrySorterUnsortedSizeGauge.WithLabelValues(captureAddr, changefeedID, tableName)
    59  	metricEntrySorterSortDuration := entrySorterSortDuration.WithLabelValues(captureAddr, changefeedID, tableName)
    60  	metricEntrySorterMergeDuration := entrySorterMergeDuration.WithLabelValues(captureAddr, changefeedID, tableName)
    61  
    62  	lessFunc := func(i *model.PolymorphicEvent, j *model.PolymorphicEvent) bool {
    63  		if i.CRTs == j.CRTs {
    64  			if i.RawKV.OpType == model.OpTypeDelete {
    65  				return true
    66  			}
    67  			if j.RawKV.OpType == model.OpTypeResolved {
    68  				return true
    69  			}
    70  		}
    71  		return i.CRTs < j.CRTs
    72  	}
    73  	mergeFunc := func(kvsA []*model.PolymorphicEvent, kvsB []*model.PolymorphicEvent, output func(*model.PolymorphicEvent)) {
    74  		var i, j int
    75  		for i < len(kvsA) && j < len(kvsB) {
    76  			if lessFunc(kvsA[i], kvsB[j]) {
    77  				output(kvsA[i])
    78  				i++
    79  			} else {
    80  				output(kvsB[j])
    81  				j++
    82  			}
    83  		}
    84  		for ; i < len(kvsA); i++ {
    85  			output(kvsA[i])
    86  		}
    87  		for ; j < len(kvsB); j++ {
    88  			output(kvsB[j])
    89  		}
    90  	}
    91  	output := func(ctx context.Context, entry *model.PolymorphicEvent) {
    92  		select {
    93  		case <-ctx.Done():
    94  			return
    95  		case es.outputCh <- entry:
    96  		}
    97  	}
    98  
    99  	errg, ctx := errgroup.WithContext(ctx)
   100  	receiver, err := es.resolvedNotifier.NewReceiver(1000 * time.Millisecond)
   101  	if err != nil {
   102  		return err
   103  	}
   104  	defer es.resolvedNotifier.Close()
   105  	errg.Go(func() error {
   106  		var sorted []*model.PolymorphicEvent
   107  		for {
   108  			select {
   109  			case <-ctx.Done():
   110  				atomic.StoreInt32(&es.closed, 1)
   111  				close(es.outputCh)
   112  				return errors.Trace(ctx.Err())
   113  			case <-time.After(defaultMetricInterval):
   114  				metricEntrySorterOutputChanSizeGauge.Set(float64(len(es.outputCh)))
   115  				es.lock.Lock()
   116  				metricEntrySorterResolvedChanSizeGuage.Set(float64(len(es.resolvedTsGroup)))
   117  				metricEntryUnsortedSizeGauge.Set(float64(len(es.unsorted)))
   118  				es.lock.Unlock()
   119  			case <-receiver.C:
   120  				es.lock.Lock()
   121  				if len(es.resolvedTsGroup) == 0 {
   122  					es.lock.Unlock()
   123  					continue
   124  				}
   125  				resolvedTsGroup := es.resolvedTsGroup
   126  				es.resolvedTsGroup = nil
   127  				toSort := es.unsorted
   128  				es.unsorted = nil
   129  				es.lock.Unlock()
   130  
   131  				resEvents := make([]*model.PolymorphicEvent, len(resolvedTsGroup))
   132  				for i, rts := range resolvedTsGroup {
   133  					// regionID = 0 means the event is produced by TiCDC
   134  					resEvents[i] = model.NewResolvedPolymorphicEvent(0, rts)
   135  				}
   136  				toSort = append(toSort, resEvents...)
   137  				startTime := time.Now()
   138  				sort.Slice(toSort, func(i, j int) bool {
   139  					return lessFunc(toSort[i], toSort[j])
   140  				})
   141  				metricEntrySorterSortDuration.Observe(time.Since(startTime).Seconds())
   142  				maxResolvedTs := resolvedTsGroup[len(resolvedTsGroup)-1]
   143  
   144  				startTime = time.Now()
   145  				var merged []*model.PolymorphicEvent
   146  				mergeFunc(toSort, sorted, func(entry *model.PolymorphicEvent) {
   147  					if entry.CRTs <= maxResolvedTs {
   148  						output(ctx, entry)
   149  					} else {
   150  						merged = append(merged, entry)
   151  					}
   152  				})
   153  				metricEntrySorterMergeDuration.Observe(time.Since(startTime).Seconds())
   154  				sorted = merged
   155  			}
   156  		}
   157  	})
   158  	return errg.Wait()
   159  }
   160  
   161  // AddEntry adds an RawKVEntry to the EntryGroup
   162  func (es *EntrySorter) AddEntry(ctx context.Context, entry *model.PolymorphicEvent) {
   163  	if atomic.LoadInt32(&es.closed) != 0 {
   164  		return
   165  	}
   166  	es.lock.Lock()
   167  	if entry.RawKV.OpType == model.OpTypeResolved {
   168  		es.resolvedTsGroup = append(es.resolvedTsGroup, entry.CRTs)
   169  		es.resolvedNotifier.Notify()
   170  	} else {
   171  		es.unsorted = append(es.unsorted, entry)
   172  	}
   173  	es.lock.Unlock()
   174  }
   175  
   176  // Output returns the sorted raw kv output channel
   177  func (es *EntrySorter) Output() <-chan *model.PolymorphicEvent {
   178  	return es.outputCh
   179  }
   180  
   181  // SortOutput receives a channel from a puller, then sort event and output to the channel returned.
   182  func SortOutput(ctx context.Context, input <-chan *model.RawKVEntry) <-chan *model.RawKVEntry {
   183  	ctx, cancel := context.WithCancel(ctx)
   184  	sorter := NewEntrySorter()
   185  	outputCh := make(chan *model.RawKVEntry, 128)
   186  	output := func(rawKV *model.RawKVEntry) {
   187  		select {
   188  		case <-ctx.Done():
   189  			if errors.Cause(ctx.Err()) != context.Canceled {
   190  				log.Error("sorter exited with error", zap.Error(ctx.Err()))
   191  			}
   192  			return
   193  		case outputCh <- rawKV:
   194  		}
   195  	}
   196  	go func() {
   197  		for {
   198  			select {
   199  			case <-ctx.Done():
   200  				if errors.Cause(ctx.Err()) != context.Canceled {
   201  					log.Error("sorter exited with error", zap.Error(ctx.Err()))
   202  				}
   203  				return
   204  			case rawKV := <-input:
   205  				if rawKV == nil {
   206  					continue
   207  				}
   208  				sorter.AddEntry(ctx, model.NewPolymorphicEvent(rawKV))
   209  			case sorted := <-sorter.Output():
   210  				if sorted != nil {
   211  					output(sorted.RawKV)
   212  				}
   213  			}
   214  		}
   215  	}()
   216  	go func() {
   217  		if err := sorter.Run(ctx); err != nil {
   218  			if errors.Cause(ctx.Err()) != context.Canceled {
   219  				log.Error("sorter exited with error", zap.Error(ctx.Err()))
   220  			}
   221  		}
   222  		cancel()
   223  	}()
   224  	return outputCh
   225  }