github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/puller/multiplexing_puller.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 puller
    15  
    16  import (
    17  	"context"
    18  	"sync"
    19  	"sync/atomic"
    20  	"time"
    21  
    22  	"github.com/pingcap/errors"
    23  	"github.com/pingcap/log"
    24  	"github.com/pingcap/tiflow/cdc/kv"
    25  	"github.com/pingcap/tiflow/cdc/model"
    26  	"github.com/pingcap/tiflow/cdc/processor/tablepb"
    27  	"github.com/pingcap/tiflow/cdc/puller/frontier"
    28  	"github.com/pingcap/tiflow/pkg/pdutil"
    29  	"github.com/pingcap/tiflow/pkg/spanz"
    30  	"github.com/prometheus/client_golang/prometheus"
    31  	"github.com/tikv/client-go/v2/oracle"
    32  	"go.uber.org/zap"
    33  	"golang.org/x/sync/errgroup"
    34  )
    35  
    36  const (
    37  	resolveLockFence        time.Duration = 4 * time.Second
    38  	resolveLockTickInterval time.Duration = 2 * time.Second
    39  
    40  	// Suppose there are 50K tables, total size of `resolvedEventsCache`s will be
    41  	// unsafe.SizeOf(kv.MultiplexingEvent) * 50K * 256 = 800M.
    42  	tableResolvedTsBufferSize int = 256
    43  
    44  	defaultPullerOutputChanSize = 128
    45  
    46  	inputChSize                = 1024
    47  	tableProgressAdvanceChSize = 128
    48  )
    49  
    50  type tableProgress struct {
    51  	changefeed      model.ChangeFeedID
    52  	client          *kv.SharedClient
    53  	spans           []tablepb.Span
    54  	subscriptionIDs []kv.SubscriptionID
    55  	startTs         model.Ts
    56  	tableName       string
    57  
    58  	initialized          atomic.Bool
    59  	resolvedTsUpdated    atomic.Int64
    60  	resolvedTs           atomic.Uint64
    61  	maxIngressResolvedTs atomic.Uint64
    62  
    63  	resolvedEventsCache chan kv.MultiplexingEvent
    64  	tsTracker           frontier.Frontier
    65  
    66  	consume struct {
    67  		// This lock is used to prevent the table progress from being
    68  		// removed while consuming events.
    69  		sync.RWMutex
    70  		removed bool
    71  		f       func(context.Context, *model.RawKVEntry, []tablepb.Span) error
    72  	}
    73  
    74  	scheduled atomic.Bool
    75  }
    76  
    77  func (p *tableProgress) handleResolvedSpans(ctx context.Context, e *model.ResolvedSpans) (err error) {
    78  	for _, resolvedSpan := range e.Spans {
    79  		if !spanz.IsSubSpan(resolvedSpan.Span, p.spans...) {
    80  			log.Panic("the resolved span is not in the table spans",
    81  				zap.String("namespace", p.changefeed.Namespace),
    82  				zap.String("changefeed", p.changefeed.ID),
    83  				zap.String("tableName", p.tableName),
    84  				zap.Any("spans", p.spans))
    85  		}
    86  		p.tsTracker.Forward(resolvedSpan.Region, resolvedSpan.Span, e.ResolvedTs)
    87  		if e.ResolvedTs > p.maxIngressResolvedTs.Load() {
    88  			p.maxIngressResolvedTs.Store(e.ResolvedTs)
    89  		}
    90  	}
    91  	resolvedTs := p.tsTracker.Frontier()
    92  
    93  	if resolvedTs > 0 && p.initialized.CompareAndSwap(false, true) {
    94  		log.Info("puller is initialized",
    95  			zap.String("namespace", p.changefeed.Namespace),
    96  			zap.String("changefeed", p.changefeed.ID),
    97  			zap.String("tableName", p.tableName),
    98  			zap.Uint64("resolvedTs", resolvedTs))
    99  	}
   100  	if resolvedTs > p.resolvedTs.Load() {
   101  		p.resolvedTs.Store(resolvedTs)
   102  		p.resolvedTsUpdated.Store(time.Now().Unix())
   103  		raw := &model.RawKVEntry{CRTs: resolvedTs, OpType: model.OpTypeResolved}
   104  		err = p.consume.f(ctx, raw, p.spans)
   105  	}
   106  
   107  	return
   108  }
   109  
   110  func (p *tableProgress) resolveLock(currentTime time.Time) {
   111  	resolvedTsUpdated := time.Unix(p.resolvedTsUpdated.Load(), 0)
   112  	if !p.initialized.Load() || time.Since(resolvedTsUpdated) < resolveLockFence {
   113  		return
   114  	}
   115  	resolvedTs := p.resolvedTs.Load()
   116  	resolvedTime := oracle.GetTimeFromTS(resolvedTs)
   117  	if currentTime.Sub(resolvedTime) < resolveLockFence {
   118  		return
   119  	}
   120  
   121  	targetTs := oracle.GoTimeToTS(resolvedTime.Add(resolveLockFence))
   122  	for _, subID := range p.subscriptionIDs {
   123  		p.client.ResolveLock(subID, targetTs)
   124  	}
   125  }
   126  
   127  type subscription struct {
   128  	*tableProgress
   129  	subID kv.SubscriptionID
   130  }
   131  
   132  // MultiplexingPuller works with `kv.SharedClient`. All tables share resources.
   133  type MultiplexingPuller struct {
   134  	changefeed model.ChangeFeedID
   135  	client     *kv.SharedClient
   136  	pdClock    pdutil.Clock
   137  	consume    func(context.Context, *model.RawKVEntry, []tablepb.Span) error
   138  	// inputChannelIndexer is used to determine which input channel to use for a given span.
   139  	inputChannelIndexer func(span tablepb.Span, workerCount int) int
   140  
   141  	// inputChs is used to collect events from client.
   142  	inputChs []chan kv.MultiplexingEvent
   143  	// tableProgressAdvanceCh is used to notify the tableAdvancer goroutine
   144  	// to advance the progress of a table.
   145  	tableProgressAdvanceCh chan *tableProgress
   146  
   147  	// NOTE: A tableProgress can have multiple subscription, all of them share
   148  	// the same tableProgress. So, we use two maps to store the relationshipxxx
   149  	// between subscription and tableProgress.
   150  	subscriptions struct {
   151  		sync.RWMutex
   152  		// m map subscriptionID -> tableProgress, used to cache and
   153  		// get tableProgress by subscriptionID quickly.
   154  		m map[kv.SubscriptionID]*tableProgress
   155  		// n map span -> subscription, used to cache and get subscription by span quickly.
   156  		n *spanz.HashMap[subscription]
   157  	}
   158  
   159  	resolvedTsAdvancerCount int
   160  
   161  	CounterKv              prometheus.Counter
   162  	CounterResolved        prometheus.Counter
   163  	CounterResolvedDropped prometheus.Counter
   164  	queueKvDuration        prometheus.Observer
   165  	queueResolvedDuration  prometheus.Observer
   166  }
   167  
   168  // NewMultiplexingPuller creates a MultiplexingPuller.
   169  // `workerCount` specifies how many workers will be spawned to handle events from kv client.
   170  // `frontierCount` specifies how many workers will be spawned to handle resolvedTs event.
   171  func NewMultiplexingPuller(
   172  	changefeed model.ChangeFeedID,
   173  	client *kv.SharedClient,
   174  	pdClock pdutil.Clock,
   175  	consume func(context.Context, *model.RawKVEntry, []tablepb.Span) error,
   176  	workerCount int,
   177  	inputChannelIndexer func(tablepb.Span, int) int,
   178  	resolvedTsAdvancerCount int,
   179  ) *MultiplexingPuller {
   180  	mpuller := &MultiplexingPuller{
   181  		changefeed:              changefeed,
   182  		client:                  client,
   183  		pdClock:                 pdClock,
   184  		consume:                 consume,
   185  		inputChannelIndexer:     inputChannelIndexer,
   186  		resolvedTsAdvancerCount: resolvedTsAdvancerCount,
   187  		tableProgressAdvanceCh:  make(chan *tableProgress, tableProgressAdvanceChSize),
   188  	}
   189  	mpuller.subscriptions.m = make(map[kv.SubscriptionID]*tableProgress)
   190  	mpuller.subscriptions.n = spanz.NewHashMap[subscription]()
   191  
   192  	mpuller.inputChs = make([]chan kv.MultiplexingEvent, 0, workerCount)
   193  	for i := 0; i < workerCount; i++ {
   194  		mpuller.inputChs = append(mpuller.inputChs, make(chan kv.MultiplexingEvent, inputChSize))
   195  	}
   196  	return mpuller
   197  }
   198  
   199  // Subscribe some spans. They will share one same resolved timestamp progress.
   200  func (p *MultiplexingPuller) Subscribe(spans []tablepb.Span, startTs model.Ts, tableName string) {
   201  	p.subscriptions.Lock()
   202  	defer p.subscriptions.Unlock()
   203  	p.subscribe(spans, startTs, tableName)
   204  }
   205  
   206  func (p *MultiplexingPuller) subscribe(
   207  	spans []tablepb.Span,
   208  	startTs model.Ts,
   209  	tableName string,
   210  ) {
   211  	for _, span := range spans {
   212  		// Base on the current design, a MultiplexingPuller is only used for one changefeed.
   213  		// So, one span can only be subscribed once.
   214  		if _, exists := p.subscriptions.n.Get(span); exists {
   215  			log.Panic("redundant subscription",
   216  				zap.String("namespace", p.changefeed.Namespace),
   217  				zap.String("changefeed", p.changefeed.ID),
   218  				zap.String("span", span.String()))
   219  		}
   220  	}
   221  
   222  	// Create a new table progress for the spans.
   223  	progress := &tableProgress{
   224  		changefeed:      p.changefeed,
   225  		client:          p.client,
   226  		spans:           spans,
   227  		subscriptionIDs: make([]kv.SubscriptionID, len(spans)),
   228  		startTs:         startTs,
   229  		tableName:       tableName,
   230  
   231  		resolvedEventsCache: make(chan kv.MultiplexingEvent, tableResolvedTsBufferSize),
   232  		tsTracker:           frontier.NewFrontier(0, spans...),
   233  	}
   234  
   235  	progress.consume.f = func(
   236  		ctx context.Context,
   237  		raw *model.RawKVEntry,
   238  		spans []tablepb.Span,
   239  	) error {
   240  		progress.consume.RLock()
   241  		defer progress.consume.RUnlock()
   242  		if !progress.consume.removed {
   243  			return p.consume(ctx, raw, spans)
   244  		}
   245  		return nil
   246  	}
   247  
   248  	for i, span := range spans {
   249  		subID := p.client.AllocSubscriptionID()
   250  		progress.subscriptionIDs[i] = subID
   251  
   252  		p.subscriptions.m[subID] = progress
   253  		p.subscriptions.n.ReplaceOrInsert(span, subscription{progress, subID})
   254  
   255  		slot := p.inputChannelIndexer(span, len(p.inputChs))
   256  		p.client.Subscribe(subID, span, startTs, p.inputChs[slot])
   257  	}
   258  
   259  	progress.initialized.Store(false)
   260  	progress.resolvedTsUpdated.Store(time.Now().Unix())
   261  }
   262  
   263  // Unsubscribe some spans, which must be subscribed in one call.
   264  func (p *MultiplexingPuller) Unsubscribe(spans []tablepb.Span) {
   265  	p.subscriptions.Lock()
   266  	defer p.subscriptions.Unlock()
   267  	p.unsubscribe(spans)
   268  }
   269  
   270  func (p *MultiplexingPuller) unsubscribe(spans []tablepb.Span) {
   271  	var progress *tableProgress
   272  	for _, span := range spans {
   273  		if prog, exists := p.subscriptions.n.Get(span); exists {
   274  			if prog.tableProgress != progress && progress != nil {
   275  				log.Panic("unsubscribe spans not in one subscription",
   276  					zap.String("namespace", p.changefeed.Namespace),
   277  					zap.String("changefeed", p.changefeed.ID))
   278  			}
   279  			progress = prog.tableProgress
   280  		} else {
   281  			log.Panic("unexist unsubscription",
   282  				zap.String("namespace", p.changefeed.Namespace),
   283  				zap.String("changefeed", p.changefeed.ID),
   284  				zap.String("span", span.String()))
   285  		}
   286  	}
   287  	if len(progress.spans) != len(spans) {
   288  		log.Panic("unsubscribe spans not same with subscription",
   289  			zap.String("namespace", p.changefeed.Namespace),
   290  			zap.String("changefeed", p.changefeed.ID))
   291  	}
   292  
   293  	progress.consume.Lock()
   294  	progress.consume.removed = true
   295  	progress.consume.Unlock()
   296  	for i, span := range progress.spans {
   297  		p.client.Unsubscribe(progress.subscriptionIDs[i])
   298  		delete(p.subscriptions.m, progress.subscriptionIDs[i])
   299  		p.subscriptions.n.Delete(span)
   300  	}
   301  }
   302  
   303  // Run the puller.
   304  func (p *MultiplexingPuller) Run(ctx context.Context) (err error) {
   305  	return p.run(ctx, true)
   306  }
   307  
   308  func (p *MultiplexingPuller) run(ctx context.Context, includeClient bool) error {
   309  	p.CounterKv = PullerEventCounter.WithLabelValues(p.changefeed.Namespace, p.changefeed.ID, "kv")
   310  	p.CounterResolved = PullerEventCounter.WithLabelValues(p.changefeed.Namespace, p.changefeed.ID, "resolved")
   311  	p.CounterResolvedDropped = PullerEventCounter.WithLabelValues(p.changefeed.Namespace, p.changefeed.ID, "resolved-dropped")
   312  	p.queueKvDuration = pullerQueueDuration.WithLabelValues(p.changefeed.Namespace, p.changefeed.ID, "kv")
   313  	p.queueResolvedDuration = pullerQueueDuration.WithLabelValues(p.changefeed.Namespace, p.changefeed.ID, "resolved")
   314  	defer func() {
   315  		PullerEventCounter.DeleteLabelValues(p.changefeed.Namespace, p.changefeed.ID, "kv")
   316  		PullerEventCounter.DeleteLabelValues(p.changefeed.Namespace, p.changefeed.ID, "resolved")
   317  		PullerEventCounter.DeleteLabelValues(p.changefeed.Namespace, p.changefeed.ID, "resolved-dropped")
   318  		pullerQueueDuration.DeleteLabelValues(p.changefeed.Namespace, p.changefeed.ID, "kv")
   319  		pullerQueueDuration.DeleteLabelValues(p.changefeed.Namespace, p.changefeed.ID, "resolved")
   320  		log.Info("MultiplexingPuller exits",
   321  			zap.String("namespace", p.changefeed.Namespace),
   322  			zap.String("changefeed", p.changefeed.ID))
   323  	}()
   324  
   325  	eg, ctx := errgroup.WithContext(ctx)
   326  
   327  	// Only !includeClient in tests.
   328  	if includeClient {
   329  		eg.Go(func() error { return p.client.Run(ctx) })
   330  	}
   331  
   332  	// Start workers to handle events received from kv client.
   333  	for i := range p.inputChs {
   334  		inputCh := p.inputChs[i]
   335  		eg.Go(func() error { return p.runEventHandler(ctx, inputCh) })
   336  	}
   337  
   338  	// Start workers to check and resolve stale locks.
   339  	eg.Go(func() error { return p.runResolveLockChecker(ctx) })
   340  
   341  	for i := 0; i < p.resolvedTsAdvancerCount; i++ {
   342  		eg.Go(func() error { return p.runResolvedTsAdvancer(ctx) })
   343  	}
   344  
   345  	log.Info("MultiplexingPuller starts",
   346  		zap.String("namespace", p.changefeed.Namespace),
   347  		zap.String("changefeed", p.changefeed.ID),
   348  		zap.Int("workerConcurrent", len(p.inputChs)),
   349  		zap.Int("frontierConcurrent", p.resolvedTsAdvancerCount))
   350  	return eg.Wait()
   351  }
   352  
   353  // runEventHandler consumes events from inputCh:
   354  // 1. If the event is a kv event, consume by calling progress.consume.f.
   355  // 2. If the event is a resolved event, send it to the resolvedEventsCache of the corresponding progress.
   356  func (p *MultiplexingPuller) runEventHandler(ctx context.Context, inputCh <-chan kv.MultiplexingEvent) error {
   357  	for {
   358  		var e kv.MultiplexingEvent
   359  		select {
   360  		case <-ctx.Done():
   361  			return ctx.Err()
   362  		case e = <-inputCh:
   363  		}
   364  
   365  		progress := p.getProgress(e.SubscriptionID)
   366  		// There is a chance that some stale events are received after
   367  		// the subscription is removed. We can just ignore them.
   368  		if progress == nil {
   369  			continue
   370  		}
   371  
   372  		if e.Val != nil {
   373  			p.queueKvDuration.Observe(float64(time.Since(e.Start).Milliseconds()))
   374  			p.CounterKv.Inc()
   375  			if err := progress.consume.f(ctx, e.Val, progress.spans); err != nil {
   376  				return errors.Trace(err)
   377  			}
   378  		} else if e.Resolved != nil {
   379  			p.CounterResolved.Add(float64(len(e.Resolved.Spans)))
   380  			select {
   381  			case <-ctx.Done():
   382  				return ctx.Err()
   383  			case progress.resolvedEventsCache <- e:
   384  				p.schedule(ctx, progress)
   385  			default:
   386  				p.CounterResolvedDropped.Add(float64(len(e.Resolved.Spans)))
   387  			}
   388  		}
   389  	}
   390  }
   391  
   392  func (p *MultiplexingPuller) getProgress(subID kv.SubscriptionID) *tableProgress {
   393  	p.subscriptions.RLock()
   394  	defer p.subscriptions.RUnlock()
   395  	return p.subscriptions.m[subID]
   396  }
   397  
   398  func (p *MultiplexingPuller) getAllProgresses() map[*tableProgress]struct{} {
   399  	p.subscriptions.RLock()
   400  	defer p.subscriptions.RUnlock()
   401  	hashset := make(map[*tableProgress]struct{}, len(p.subscriptions.m))
   402  	for _, value := range p.subscriptions.m {
   403  		hashset[value] = struct{}{}
   404  	}
   405  	return hashset
   406  }
   407  
   408  func (p *MultiplexingPuller) schedule(ctx context.Context, progress *tableProgress) {
   409  	if progress.scheduled.CompareAndSwap(false, true) {
   410  		select {
   411  		case <-ctx.Done():
   412  		case p.tableProgressAdvanceCh <- progress:
   413  		}
   414  	}
   415  }
   416  
   417  // runResolvedTsAdvancer receives tableProgress from tableProgressAdvanceCh
   418  // and advances the resolvedTs of the tableProgress.
   419  func (p *MultiplexingPuller) runResolvedTsAdvancer(ctx context.Context) error {
   420  	advanceTableProgress := func(ctx context.Context, progress *tableProgress) error {
   421  		defer func() {
   422  			progress.scheduled.Store(false)
   423  			// Schedule the progress again if there are still events in the cache.
   424  			if len(progress.resolvedEventsCache) > 0 {
   425  				p.schedule(ctx, progress)
   426  			}
   427  		}()
   428  
   429  		var event kv.MultiplexingEvent
   430  		var spans *model.ResolvedSpans
   431  		for i := 0; i < 128; i++ {
   432  			select {
   433  			case <-ctx.Done():
   434  				return ctx.Err()
   435  			case event = <-progress.resolvedEventsCache:
   436  				spans = event.RegionFeedEvent.Resolved
   437  			default:
   438  				return nil
   439  			}
   440  			p.queueResolvedDuration.Observe(float64(time.Since(event.Start).Milliseconds()))
   441  			if err := progress.handleResolvedSpans(ctx, spans); err != nil {
   442  				return errors.Trace(err)
   443  			}
   444  		}
   445  		return nil
   446  	}
   447  
   448  	var progress *tableProgress
   449  	for {
   450  		select {
   451  		case <-ctx.Done():
   452  			return ctx.Err()
   453  		case progress = <-p.tableProgressAdvanceCh:
   454  			if err := advanceTableProgress(ctx, progress); err != nil {
   455  				return errors.Trace(err)
   456  			}
   457  		}
   458  	}
   459  }
   460  
   461  func (p *MultiplexingPuller) runResolveLockChecker(ctx context.Context) error {
   462  	resolveLockTicker := time.NewTicker(resolveLockTickInterval)
   463  	defer resolveLockTicker.Stop()
   464  	for {
   465  		select {
   466  		case <-ctx.Done():
   467  			return ctx.Err()
   468  		case <-resolveLockTicker.C:
   469  		}
   470  		currentTime := p.pdClock.CurrentTime()
   471  		for progress := range p.getAllProgresses() {
   472  			select {
   473  			case <-ctx.Done():
   474  				return ctx.Err()
   475  			default:
   476  				progress.resolveLock(currentTime)
   477  			}
   478  		}
   479  	}
   480  }
   481  
   482  // Stats of a puller.
   483  type Stats struct {
   484  	RegionCount         uint64
   485  	CheckpointTsIngress model.Ts
   486  	ResolvedTsIngress   model.Ts
   487  	CheckpointTsEgress  model.Ts
   488  	ResolvedTsEgress    model.Ts
   489  }
   490  
   491  // Stats returns Stats.
   492  func (p *MultiplexingPuller) Stats(span tablepb.Span) Stats {
   493  	p.subscriptions.RLock()
   494  	progress := p.subscriptions.n.GetV(span)
   495  	p.subscriptions.RUnlock()
   496  	if progress.tableProgress == nil {
   497  		return Stats{}
   498  	}
   499  	return Stats{
   500  		RegionCount:         p.client.RegionCount(progress.subID),
   501  		ResolvedTsIngress:   progress.maxIngressResolvedTs.Load(),
   502  		CheckpointTsIngress: progress.maxIngressResolvedTs.Load(),
   503  		ResolvedTsEgress:    progress.resolvedTs.Load(),
   504  		CheckpointTsEgress:  progress.resolvedTs.Load(),
   505  	}
   506  }