github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/pkg/pipeline/pipeline.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 pipeline
    15  
    16  import (
    17  	"sync"
    18  	"time"
    19  
    20  	"github.com/pingcap/failpoint"
    21  	"github.com/pingcap/log"
    22  	"github.com/pingcap/ticdc/pkg/context"
    23  	cerror "github.com/pingcap/ticdc/pkg/errors"
    24  	"go.uber.org/zap"
    25  )
    26  
    27  // Pipeline represents a pipeline includes a number of nodes
    28  type Pipeline struct {
    29  	header    headRunner
    30  	runners   []runner
    31  	runnersWg sync.WaitGroup
    32  	closeMu   sync.Mutex
    33  	isClosed  bool
    34  
    35  	outputChSize int
    36  }
    37  
    38  // NewPipeline creates a new pipeline
    39  func NewPipeline(
    40  	ctx context.Context, tickDuration time.Duration, initRunnerSize, outputChSize int,
    41  ) *Pipeline {
    42  	header := make(headRunner, 4)
    43  	runners := make([]runner, 0, initRunnerSize)
    44  	runners = append(runners, header)
    45  	p := &Pipeline{
    46  		header:       header,
    47  		runners:      runners,
    48  		outputChSize: outputChSize,
    49  	}
    50  	go func() {
    51  		var tickCh <-chan time.Time
    52  		if tickDuration > 0 {
    53  			ticker := time.NewTicker(tickDuration)
    54  			defer ticker.Stop()
    55  			tickCh = ticker.C
    56  		} else {
    57  			tickCh = make(chan time.Time)
    58  		}
    59  		for {
    60  			select {
    61  			case <-tickCh:
    62  				err := p.SendToFirstNode(TickMessage()) //nolint:errcheck
    63  				if err != nil {
    64  					// Errors here are innocent. It's okay for tick messages to get lost.
    65  					log.Debug("Error encountered when calling SendToFirstNode", zap.Error(err))
    66  				}
    67  			case <-ctx.Done():
    68  				p.close()
    69  				return
    70  			}
    71  		}
    72  	}()
    73  	return p
    74  }
    75  
    76  // AppendNode appends the node to the pipeline
    77  func (p *Pipeline) AppendNode(ctx context.Context, name string, node Node) {
    78  	ctx = context.WithErrorHandler(ctx, func(err error) error {
    79  		p.close()
    80  		return err
    81  	})
    82  	lastRunner := p.runners[len(p.runners)-1]
    83  	runner := newNodeRunner(name, node, lastRunner, p.outputChSize)
    84  	p.runners = append(p.runners, runner)
    85  	p.runnersWg.Add(1)
    86  	go p.driveRunner(ctx, lastRunner, runner)
    87  }
    88  
    89  func (p *Pipeline) driveRunner(ctx context.Context, previousRunner, runner runner) {
    90  	defer func() {
    91  		log.Debug("a pipeline node is exiting, stop the whole pipeline", zap.String("name", runner.getName()))
    92  		p.close()
    93  		blackhole(previousRunner)
    94  		p.runnersWg.Done()
    95  	}()
    96  	err := runner.run(ctx)
    97  	if err != nil {
    98  		ctx.Throw(err)
    99  		if cerror.ErrTableProcessorStoppedSafely.NotEqual(err) {
   100  			log.Error("found error when running the node", zap.String("name", runner.getName()), zap.Error(err))
   101  		}
   102  	}
   103  }
   104  
   105  var pipelineTryAgainError error = cerror.ErrPipelineTryAgain.FastGenByArgs()
   106  
   107  // SendToFirstNode sends the message to the first node
   108  func (p *Pipeline) SendToFirstNode(msg Message) error {
   109  	p.closeMu.Lock()
   110  	defer p.closeMu.Unlock()
   111  	if p.isClosed {
   112  		return cerror.ErrSendToClosedPipeline.GenWithStackByArgs()
   113  	}
   114  
   115  	failpoint.Inject("PipelineSendToFirstNodeTryAgain", func() {
   116  		failpoint.Return(cerror.ErrPipelineTryAgain.GenWithStackByArgs())
   117  	})
   118  
   119  	select {
   120  	case p.header <- msg:
   121  	default:
   122  		// Do not call `GenWithStackByArgs` in the hot path,
   123  		// it consumes lots of CPU.
   124  		return pipelineTryAgainError
   125  	}
   126  	return nil
   127  }
   128  
   129  func (p *Pipeline) close() {
   130  	p.closeMu.Lock()
   131  	defer p.closeMu.Unlock()
   132  	if !p.isClosed {
   133  		close(p.header)
   134  		p.isClosed = true
   135  	}
   136  }
   137  
   138  // Wait all the nodes exited
   139  func (p *Pipeline) Wait() {
   140  	p.runnersWg.Wait()
   141  }