github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/capture.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 cdc
    15  
    16  import (
    17  	"context"
    18  	"sync"
    19  	"time"
    20  
    21  	"github.com/google/uuid"
    22  	"github.com/pingcap/errors"
    23  	"github.com/pingcap/failpoint"
    24  	"github.com/pingcap/log"
    25  	"github.com/pingcap/ticdc/cdc/kv"
    26  	"github.com/pingcap/ticdc/cdc/model"
    27  	"github.com/pingcap/ticdc/cdc/processor"
    28  	"github.com/pingcap/ticdc/pkg/config"
    29  	cdcContext "github.com/pingcap/ticdc/pkg/context"
    30  	cerror "github.com/pingcap/ticdc/pkg/errors"
    31  	"github.com/pingcap/ticdc/pkg/orchestrator"
    32  	"github.com/pingcap/ticdc/pkg/util"
    33  	"github.com/pingcap/ticdc/pkg/version"
    34  	tidbkv "github.com/pingcap/tidb/kv"
    35  	pd "github.com/tikv/pd/client"
    36  	"go.etcd.io/etcd/clientv3"
    37  	"go.etcd.io/etcd/clientv3/concurrency"
    38  	"go.etcd.io/etcd/mvcc"
    39  	"go.etcd.io/etcd/pkg/logutil"
    40  	"go.uber.org/zap"
    41  	"go.uber.org/zap/zapcore"
    42  	"google.golang.org/grpc"
    43  	"google.golang.org/grpc/backoff"
    44  )
    45  
    46  // Capture represents a Capture server, it monitors the changefeed information in etcd and schedules Task on it.
    47  type Capture struct {
    48  	etcdClient kv.CDCEtcdClient
    49  	pdCli      pd.Client
    50  	kvStorage  tidbkv.Storage
    51  	grpcPool   kv.GrpcPool
    52  
    53  	processorManager *processor.Manager
    54  
    55  	processors map[string]*oldProcessor
    56  	procLock   sync.Mutex
    57  
    58  	info *model.CaptureInfo
    59  
    60  	// session keeps alive between the capture and etcd
    61  	session  *concurrency.Session
    62  	election *concurrency.Election
    63  
    64  	closed chan struct{}
    65  }
    66  
    67  // NewCapture returns a new Capture instance
    68  func NewCapture(
    69  	stdCtx context.Context,
    70  	pdEndpoints []string,
    71  	pdCli pd.Client,
    72  	kvStorage tidbkv.Storage,
    73  ) (c *Capture, err error) {
    74  	conf := config.GetGlobalServerConfig()
    75  	credential := conf.Security
    76  	tlsConfig, err := credential.ToTLSConfig()
    77  	if err != nil {
    78  		return nil, errors.Trace(err)
    79  	}
    80  	grpcTLSOption, err := credential.ToGRPCDialOption()
    81  	if err != nil {
    82  		return nil, errors.Trace(err)
    83  	}
    84  	logConfig := logutil.DefaultZapLoggerConfig
    85  	logConfig.Level = zap.NewAtomicLevelAt(zapcore.ErrorLevel)
    86  	etcdCli, err := clientv3.New(clientv3.Config{
    87  		Endpoints:   pdEndpoints,
    88  		TLS:         tlsConfig,
    89  		Context:     stdCtx,
    90  		LogConfig:   &logConfig,
    91  		DialTimeout: 5 * time.Second,
    92  		DialOptions: []grpc.DialOption{
    93  			grpcTLSOption,
    94  			grpc.WithBlock(),
    95  			grpc.WithConnectParams(grpc.ConnectParams{
    96  				Backoff: backoff.Config{
    97  					BaseDelay:  time.Second,
    98  					Multiplier: 1.1,
    99  					Jitter:     0.1,
   100  					MaxDelay:   3 * time.Second,
   101  				},
   102  				MinConnectTimeout: 3 * time.Second,
   103  			}),
   104  		},
   105  	})
   106  	if err != nil {
   107  		return nil, errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "new etcd client")
   108  	}
   109  	sess, err := concurrency.NewSession(etcdCli,
   110  		concurrency.WithTTL(conf.CaptureSessionTTL))
   111  	if err != nil {
   112  		return nil, errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "create capture session")
   113  	}
   114  	elec := concurrency.NewElection(sess, kv.CaptureOwnerKey)
   115  	cli := kv.NewCDCEtcdClient(stdCtx, etcdCli)
   116  	id := uuid.New().String()
   117  	info := &model.CaptureInfo{
   118  		ID:            id,
   119  		AdvertiseAddr: conf.AdvertiseAddr,
   120  		Version:       version.ReleaseVersion,
   121  	}
   122  	processorManager := processor.NewManager()
   123  	grpcPool := kv.NewGrpcPoolImpl(stdCtx, credential)
   124  	log.Info("creating capture", zap.String("capture-id", id), util.ZapFieldCapture(stdCtx))
   125  
   126  	c = &Capture{
   127  		processors:       make(map[string]*oldProcessor),
   128  		etcdClient:       cli,
   129  		grpcPool:         grpcPool,
   130  		session:          sess,
   131  		election:         elec,
   132  		info:             info,
   133  		pdCli:            pdCli,
   134  		kvStorage:        kvStorage,
   135  		processorManager: processorManager,
   136  		closed:           make(chan struct{}),
   137  	}
   138  
   139  	return
   140  }
   141  
   142  // Run runs the Capture mainloop
   143  func (c *Capture) Run(ctx context.Context) (err error) {
   144  	ctx, cancel := context.WithCancel(ctx)
   145  	// TODO: we'd better to add some wait mechanism to ensure no routine is blocked
   146  	defer cancel()
   147  	defer close(c.closed)
   148  
   149  	ctx = cdcContext.NewContext(ctx, &cdcContext.GlobalVars{
   150  		PDClient:    c.pdCli,
   151  		KVStorage:   c.kvStorage,
   152  		CaptureInfo: c.info,
   153  	})
   154  	err = c.register(ctx)
   155  	if err != nil {
   156  		return errors.Trace(err)
   157  	}
   158  	if config.NewReplicaImpl {
   159  		sessionCli := c.session.Client()
   160  		etcdWorker, err := orchestrator.NewEtcdWorker(kv.NewCDCEtcdClient(ctx, sessionCli).Client, kv.EtcdKeyBase, c.processorManager, model.NewGlobalState())
   161  		if err != nil {
   162  			return errors.Trace(err)
   163  		}
   164  		log.Info("start to listen processor task...")
   165  		if err := etcdWorker.Run(ctx, c.session, 200*time.Millisecond); err != nil {
   166  			// We check ttl of lease instead of check `session.Done`, because
   167  			// `session.Done` is only notified when etcd client establish a
   168  			// new keepalive request, there could be a time window as long as
   169  			// 1/3 of session ttl that `session.Done` can't be triggered even
   170  			// the lease is already revoked.
   171  			if cerror.ErrEtcdSessionDone.Equal(err) {
   172  				log.Warn("session is disconnected", zap.Error(err))
   173  				return cerror.ErrCaptureSuicide.GenWithStackByArgs()
   174  			}
   175  			lease, inErr := c.etcdClient.Client.TimeToLive(ctx, c.session.Lease())
   176  			if inErr != nil {
   177  				return cerror.WrapError(cerror.ErrPDEtcdAPIError, inErr)
   178  			}
   179  			if lease.TTL == int64(-1) {
   180  				log.Warn("handle task event failed because session is disconnected", zap.Error(err))
   181  				return cerror.ErrCaptureSuicide.GenWithStackByArgs()
   182  			}
   183  			return errors.Trace(err)
   184  		}
   185  	} else {
   186  		defer c.grpcPool.Close()
   187  		go func() {
   188  			c.grpcPool.RecycleConn(ctx)
   189  		}()
   190  		taskWatcher := NewTaskWatcher(c, &TaskWatcherConfig{
   191  			Prefix:      kv.TaskStatusKeyPrefix + "/" + c.info.ID,
   192  			ChannelSize: 128,
   193  		})
   194  		log.Info("waiting for tasks", zap.String("capture-id", c.info.ID))
   195  		var ev *TaskEvent
   196  		wch := taskWatcher.Watch(ctx)
   197  		for {
   198  			// Return error when the session is done unexpectedly, it means the
   199  			// server does not send heartbeats in time, or network interrupted
   200  			// In this case, the state of the capture is undermined, the task may
   201  			// have or have not been rebalanced, the owner may be or not be held,
   202  			// so we must cancel context to let all sub routines exit.
   203  			select {
   204  			case <-c.session.Done():
   205  				if ctx.Err() != context.Canceled {
   206  					log.Info("capture session done, capture suicide itself", zap.String("capture-id", c.info.ID))
   207  					return cerror.ErrCaptureSuicide.GenWithStackByArgs()
   208  				}
   209  			case ev = <-wch:
   210  				if ev == nil {
   211  					return nil
   212  				}
   213  				if ev.Err != nil {
   214  					return errors.Trace(ev.Err)
   215  				}
   216  				failpoint.Inject("captureHandleTaskDelay", nil)
   217  				if err := c.handleTaskEvent(ctx, ev); err != nil {
   218  					// We check ttl of lease instead of check `session.Done`, because
   219  					// `session.Done` is only notified when etcd client establish a
   220  					// new keepalive request, there could be a time window as long as
   221  					// 1/3 of session ttl that `session.Done` can't be triggered even
   222  					// the lease is already revoked.
   223  					lease, inErr := c.etcdClient.Client.TimeToLive(ctx, c.session.Lease())
   224  					if inErr != nil {
   225  						return cerror.WrapError(cerror.ErrPDEtcdAPIError, inErr)
   226  					}
   227  					if lease.TTL == int64(-1) {
   228  						log.Warn("handle task event failed because session is disconnected", zap.Error(err))
   229  						return cerror.ErrCaptureSuicide.GenWithStackByArgs()
   230  					}
   231  					return errors.Trace(err)
   232  				}
   233  			}
   234  		}
   235  	}
   236  	return nil
   237  }
   238  
   239  // Campaign to be an owner
   240  func (c *Capture) Campaign(ctx context.Context) error {
   241  	failpoint.Inject("capture-campaign-compacted-error", func() {
   242  		failpoint.Return(errors.Trace(mvcc.ErrCompacted))
   243  	})
   244  	return cerror.WrapError(cerror.ErrCaptureCampaignOwner, c.election.Campaign(ctx, c.info.ID))
   245  }
   246  
   247  // Resign lets a owner start a new election.
   248  func (c *Capture) Resign(ctx context.Context) error {
   249  	failpoint.Inject("capture-resign-failed", func() {
   250  		failpoint.Return(errors.New("capture resign failed"))
   251  	})
   252  	return cerror.WrapError(cerror.ErrCaptureResignOwner, c.election.Resign(ctx))
   253  }
   254  
   255  // Cleanup cleans all dynamic resources
   256  func (c *Capture) Cleanup() {
   257  	c.procLock.Lock()
   258  	defer c.procLock.Unlock()
   259  
   260  	for _, processor := range c.processors {
   261  		processor.wait()
   262  	}
   263  }
   264  
   265  // Close closes the capture by unregistering it from etcd
   266  func (c *Capture) Close(ctx context.Context) error {
   267  	if config.NewReplicaImpl {
   268  		c.processorManager.AsyncClose()
   269  		select {
   270  		case <-c.closed:
   271  		case <-ctx.Done():
   272  		}
   273  	} else {
   274  		if c.grpcPool != nil {
   275  			c.grpcPool.Close()
   276  		}
   277  	}
   278  	return errors.Trace(c.etcdClient.DeleteCaptureInfo(ctx, c.info.ID))
   279  }
   280  
   281  func (c *Capture) handleTaskEvent(ctx context.Context, ev *TaskEvent) error {
   282  	task := ev.Task
   283  	if ev.Op == TaskOpCreate {
   284  		if _, ok := c.processors[task.ChangeFeedID]; !ok {
   285  			p, err := c.assignTask(ctx, task)
   286  			if err != nil {
   287  				return err
   288  			}
   289  			c.processors[task.ChangeFeedID] = p
   290  		}
   291  	} else if ev.Op == TaskOpDelete {
   292  		if p, ok := c.processors[task.ChangeFeedID]; ok {
   293  			if err := p.stop(ctx); err != nil {
   294  				return errors.Trace(err)
   295  			}
   296  			delete(c.processors, task.ChangeFeedID)
   297  		}
   298  	}
   299  	return nil
   300  }
   301  
   302  func (c *Capture) assignTask(ctx context.Context, task *Task) (*oldProcessor, error) {
   303  	cf, err := c.etcdClient.GetChangeFeedInfo(ctx, task.ChangeFeedID)
   304  	if err != nil {
   305  		log.Error("get change feed info failed",
   306  			zap.String("changefeed", task.ChangeFeedID),
   307  			zap.String("capture-id", c.info.ID),
   308  			util.ZapFieldCapture(ctx),
   309  			zap.Error(err))
   310  		return nil, err
   311  	}
   312  	err = cf.VerifyAndFix()
   313  	if err != nil {
   314  		return nil, err
   315  	}
   316  	log.Info("run processor",
   317  		zap.String("capture-id", c.info.ID), util.ZapFieldCapture(ctx),
   318  		zap.String("changefeed", task.ChangeFeedID))
   319  	conf := config.GetGlobalServerConfig()
   320  	p, err := runProcessorImpl(
   321  		ctx, c.pdCli, c.grpcPool, c.session, *cf, task.ChangeFeedID, *c.info, task.CheckpointTS, time.Duration(conf.ProcessorFlushInterval))
   322  	if err != nil {
   323  		log.Error("run processor failed",
   324  			zap.String("changefeed", task.ChangeFeedID),
   325  			zap.String("capture-id", c.info.ID),
   326  			util.ZapFieldCapture(ctx),
   327  			zap.Error(err))
   328  		return nil, err
   329  	}
   330  	return p, nil
   331  }
   332  
   333  // register registers the capture information in etcd
   334  func (c *Capture) register(ctx context.Context) error {
   335  	err := c.etcdClient.PutCaptureInfo(ctx, c.info, c.session.Lease())
   336  	return cerror.WrapError(cerror.ErrCaptureRegister, err)
   337  }