github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/api/v2/changefeed.go (about)

     1  // Copyright 2022 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 v2
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"net/http"
    20  	"net/url"
    21  	"sort"
    22  	"strings"
    23  	"time"
    24  
    25  	"github.com/gin-gonic/gin"
    26  	"github.com/pingcap/errors"
    27  	"github.com/pingcap/log"
    28  	tidbkv "github.com/pingcap/tidb/pkg/kv"
    29  	"github.com/pingcap/tiflow/cdc/api"
    30  	"github.com/pingcap/tiflow/cdc/capture"
    31  	"github.com/pingcap/tiflow/cdc/model"
    32  	"github.com/pingcap/tiflow/pkg/config"
    33  	cerror "github.com/pingcap/tiflow/pkg/errors"
    34  	"github.com/pingcap/tiflow/pkg/retry"
    35  	"github.com/pingcap/tiflow/pkg/txnutil/gc"
    36  	"github.com/pingcap/tiflow/pkg/upstream"
    37  	"github.com/pingcap/tiflow/pkg/util"
    38  	"github.com/tikv/client-go/v2/oracle"
    39  	pd "github.com/tikv/pd/client"
    40  	clientv3 "go.etcd.io/etcd/client/v3"
    41  	"go.uber.org/zap"
    42  )
    43  
    44  const (
    45  	// timeout for pd client
    46  	timeout = 30 * time.Second
    47  )
    48  
    49  // createChangefeed handles create changefeed request,
    50  // it returns the changefeed's changefeedInfo that it just created
    51  // CreateChangefeed creates a changefeed
    52  // @Summary Create changefeed
    53  // @Description create a new changefeed
    54  // @Tags changefeed,v2
    55  // @Accept json
    56  // @Produce json
    57  // @Param changefeed body ChangefeedConfig true "changefeed config"
    58  // @Success 200 {object} ChangeFeedInfo
    59  // @Failure 500,400 {object} model.HTTPError
    60  // @Router	/api/v2/changefeeds [post]
    61  func (h *OpenAPIV2) createChangefeed(c *gin.Context) {
    62  	ctx := c.Request.Context()
    63  	cfg := &ChangefeedConfig{ReplicaConfig: GetDefaultReplicaConfig()}
    64  
    65  	if err := c.BindJSON(&cfg); err != nil {
    66  		_ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err))
    67  		return
    68  	}
    69  	if len(cfg.PDAddrs) == 0 {
    70  		up, err := getCaptureDefaultUpstream(h.capture)
    71  		if err != nil {
    72  			_ = c.Error(err)
    73  			return
    74  		}
    75  		cfg.PDConfig = getUpstreamPDConfig(up)
    76  	}
    77  	credential := cfg.PDConfig.toCredential()
    78  
    79  	timeoutCtx, cancel := context.WithTimeout(ctx, 30*time.Second)
    80  	defer cancel()
    81  	pdClient, err := h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential)
    82  	if err != nil {
    83  		_ = c.Error(cerror.WrapError(cerror.ErrAPIGetPDClientFailed, err))
    84  		return
    85  	}
    86  	defer pdClient.Close()
    87  
    88  	// verify tables todo: del kvstore
    89  	kvStorage, err := h.helpers.createTiStore(cfg.PDAddrs, credential)
    90  	if err != nil {
    91  		_ = c.Error(cerror.WrapError(cerror.ErrNewStore, err))
    92  		return
    93  	}
    94  	ctrl, err := h.capture.GetController()
    95  	if err != nil {
    96  		_ = c.Error(err)
    97  		return
    98  	}
    99  
   100  	// We should not close kvStorage since all kvStorage in cdc is the same one.
   101  	// defer kvStorage.Close()
   102  	// TODO: We should get a kvStorage from upstream instead of creating a new one
   103  	info, err := h.helpers.verifyCreateChangefeedConfig(
   104  		ctx,
   105  		cfg,
   106  		pdClient,
   107  		ctrl,
   108  		h.capture.GetEtcdClient().GetEnsureGCServiceID(gc.EnsureGCServiceCreating),
   109  		kvStorage)
   110  	if err != nil {
   111  		_ = c.Error(err)
   112  		return
   113  	}
   114  	needRemoveGCSafePoint := false
   115  	defer func() {
   116  		if !needRemoveGCSafePoint {
   117  			return
   118  		}
   119  		err := gc.UndoEnsureChangefeedStartTsSafety(
   120  			ctx,
   121  			pdClient,
   122  			h.capture.GetEtcdClient().GetEnsureGCServiceID(gc.EnsureGCServiceCreating),
   123  			model.ChangeFeedID{Namespace: cfg.Namespace, ID: cfg.ID},
   124  		)
   125  		if err != nil {
   126  			_ = c.Error(err)
   127  			return
   128  		}
   129  	}()
   130  	upstreamInfo := &model.UpstreamInfo{
   131  		ID:            info.UpstreamID,
   132  		PDEndpoints:   strings.Join(cfg.PDAddrs, ","),
   133  		KeyPath:       cfg.KeyPath,
   134  		CertPath:      cfg.CertPath,
   135  		CAPath:        cfg.CAPath,
   136  		CertAllowedCN: cfg.CertAllowedCN,
   137  	}
   138  
   139  	// cannot create changefeed if there are running lightning/restore tasks
   140  	tlsCfg, err := credential.ToTLSConfig()
   141  	if err != nil {
   142  		_ = c.Error(err)
   143  		return
   144  	}
   145  
   146  	cli, err := h.helpers.getEtcdClient(cfg.PDAddrs, tlsCfg)
   147  	if err != nil {
   148  		_ = c.Error(err)
   149  		return
   150  	}
   151  	err = hasRunningImport(ctx, cli)
   152  	if err != nil {
   153  		log.Error("failed to create changefeed", zap.Error(err))
   154  		_ = c.Error(
   155  			cerror.ErrUpstreamHasRunningImport.Wrap(err).
   156  				FastGenByArgs(info.UpstreamID),
   157  		)
   158  		return
   159  	}
   160  
   161  	err = ctrl.CreateChangefeed(ctx,
   162  		upstreamInfo,
   163  		info)
   164  	if err != nil {
   165  		needRemoveGCSafePoint = true
   166  		_ = c.Error(err)
   167  		return
   168  	}
   169  
   170  	log.Info("Create changefeed successfully!",
   171  		zap.String("id", info.ID),
   172  		zap.String("changefeed", info.String()))
   173  	c.JSON(http.StatusOK, toAPIModel(info,
   174  		info.StartTs, info.StartTs,
   175  		nil, true))
   176  }
   177  
   178  // hasRunningImport checks if there is running import tasks on the
   179  // upstream cluster.
   180  func hasRunningImport(ctx context.Context, cli *clientv3.Client) error {
   181  	resp, err := cli.KV.Get(
   182  		ctx, RegisterImportTaskPrefix, clientv3.WithPrefix(),
   183  	)
   184  	if err != nil {
   185  		return errors.Annotatef(
   186  			err, "failed to list import task related entries")
   187  	}
   188  
   189  	for _, kv := range resp.Kvs {
   190  		leaseResp, err := cli.Lease.TimeToLive(ctx, clientv3.LeaseID(kv.Lease))
   191  		if err != nil {
   192  			return errors.Annotatef(
   193  				err, "failed to get time-to-live of lease: %x", kv.Lease,
   194  			)
   195  		}
   196  		// the lease has expired
   197  		if leaseResp.TTL <= 0 {
   198  			continue
   199  		}
   200  
   201  		err = errors.New(
   202  			"There are lightning/restore tasks running" +
   203  				"please stop or wait for them to finish. " +
   204  				"If the task is terminated by system, " +
   205  				"please wait until the task lease ttl(3 mins) decreases to 0.",
   206  		)
   207  		return err
   208  	}
   209  
   210  	return nil
   211  }
   212  
   213  // listChangeFeeds lists all changgefeeds in cdc cluster
   214  // @Summary List changefeed
   215  // @Description list all changefeeds in cdc cluster
   216  // @Tags changefeed,v2
   217  // @Accept json
   218  // @Produce json
   219  // @Param state query string false "state"
   220  // @Param namespace query string false "default"
   221  // @Success 200 {array} ChangefeedCommonInfo
   222  // @Failure 500 {object} model.HTTPError
   223  // @Router /api/v2/changefeeds [get]
   224  func (h *OpenAPIV2) listChangeFeeds(c *gin.Context) {
   225  	ctx := c.Request.Context()
   226  	state := c.Query(api.APIOpVarChangefeedState)
   227  	controller, err := h.capture.GetController()
   228  	if err != nil {
   229  		_ = c.Error(err)
   230  		return
   231  	}
   232  	checkpointTs, err := controller.GetAllChangeFeedCheckpointTs(ctx)
   233  	if err != nil {
   234  		_ = c.Error(err)
   235  		return
   236  	}
   237  	namespace := getNamespaceValueWithDefault(c)
   238  
   239  	infos, err := controller.GetAllChangeFeedInfo(ctx)
   240  	if err != nil {
   241  		_ = c.Error(err)
   242  		return
   243  	}
   244  
   245  	commonInfos := make([]ChangefeedCommonInfo, 0)
   246  	changefeeds := make([]model.ChangeFeedID, 0)
   247  
   248  	for cfID := range infos {
   249  		// filter by namespace
   250  		if cfID.Namespace == namespace {
   251  			changefeeds = append(changefeeds, cfID)
   252  		}
   253  	}
   254  	sort.Slice(changefeeds, func(i, j int) bool {
   255  		if changefeeds[i].Namespace == changefeeds[j].Namespace {
   256  			return changefeeds[i].ID < changefeeds[j].ID
   257  		}
   258  
   259  		return changefeeds[i].Namespace < changefeeds[j].Namespace
   260  	})
   261  
   262  	for _, cfID := range changefeeds {
   263  		cfInfo, exist := infos[cfID]
   264  		if !exist {
   265  			continue
   266  		}
   267  		changefeedCheckpointTs, ok := checkpointTs[cfID]
   268  
   269  		if !cfInfo.State.IsNeeded(state) {
   270  			// if the value of `state` is not 'all', only return changefeed
   271  			// with state 'normal', 'stopped', 'failed'
   272  			continue
   273  		}
   274  
   275  		// return the common info only.
   276  		commonInfo := &ChangefeedCommonInfo{
   277  			UpstreamID: cfInfo.UpstreamID,
   278  			Namespace:  cfID.Namespace,
   279  			ID:         cfID.ID,
   280  			FeedState:  cfInfo.State,
   281  		}
   282  
   283  		if cfInfo.Error != nil {
   284  			commonInfo.RunningError = cfInfo.Error
   285  		} else {
   286  			commonInfo.RunningError = cfInfo.Warning
   287  		}
   288  
   289  		// if the state is normal, we shall not return the error info
   290  		// because changefeed will is retrying. errors will confuse the users
   291  		if commonInfo.FeedState == model.StateNormal {
   292  			commonInfo.RunningError = nil
   293  		}
   294  
   295  		if ok {
   296  			commonInfo.CheckpointTSO = changefeedCheckpointTs
   297  			tm := oracle.GetTimeFromTS(changefeedCheckpointTs)
   298  			commonInfo.CheckpointTime = model.JSONTime(tm)
   299  		}
   300  
   301  		commonInfos = append(commonInfos, *commonInfo)
   302  	}
   303  	resp := &ListResponse[ChangefeedCommonInfo]{
   304  		Total: len(commonInfos),
   305  		Items: commonInfos,
   306  	}
   307  
   308  	c.JSON(http.StatusOK, resp)
   309  }
   310  
   311  func getNamespaceValueWithDefault(c *gin.Context) string {
   312  	namespace := c.Query(api.APIOpVarNamespace)
   313  	if namespace == "" {
   314  		namespace = model.DefaultNamespace
   315  	}
   316  	return namespace
   317  }
   318  
   319  // verifyTable verify table, return ineligibleTables and EligibleTables.
   320  func (h *OpenAPIV2) verifyTable(c *gin.Context) {
   321  	cfg := getDefaultVerifyTableConfig()
   322  	if err := c.BindJSON(cfg); err != nil {
   323  		_ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err))
   324  		return
   325  	}
   326  	if len(cfg.PDAddrs) == 0 {
   327  		up, err := getCaptureDefaultUpstream(h.capture)
   328  		if err != nil {
   329  			_ = c.Error(err)
   330  			return
   331  		}
   332  		cfg.PDConfig = getUpstreamPDConfig(up)
   333  	}
   334  	credential := cfg.PDConfig.toCredential()
   335  
   336  	kvStore, err := h.helpers.createTiStore(cfg.PDAddrs, credential)
   337  	if err != nil {
   338  		_ = c.Error(err)
   339  		return
   340  	}
   341  	uri, err := url.Parse(cfg.SinkURI)
   342  	if err != nil {
   343  		_ = c.Error(err)
   344  		return
   345  	}
   346  	scheme := uri.Scheme
   347  	topic := strings.TrimFunc(uri.Path, func(r rune) bool {
   348  		return r == '/'
   349  	})
   350  	replicaCfg := cfg.ReplicaConfig.ToInternalReplicaConfig()
   351  	protocol, _ := config.ParseSinkProtocolFromString(util.GetOrZero(replicaCfg.Sink.Protocol))
   352  
   353  	ineligibleTables, eligibleTables, err := h.helpers.
   354  		getVerifiedTables(replicaCfg, kvStore, cfg.StartTs, scheme, topic, protocol)
   355  	if err != nil {
   356  		_ = c.Error(err)
   357  		return
   358  	}
   359  	toAPIModelFunc := func(tbls []model.TableName) []TableName {
   360  		var apiModles []TableName
   361  		for _, tbl := range tbls {
   362  			apiModles = append(apiModles, TableName{
   363  				Schema:      tbl.Schema,
   364  				Table:       tbl.Table,
   365  				TableID:     tbl.TableID,
   366  				IsPartition: tbl.IsPartition,
   367  			})
   368  		}
   369  		return apiModles
   370  	}
   371  	tables := &Tables{
   372  		IneligibleTables: toAPIModelFunc(ineligibleTables),
   373  		EligibleTables:   toAPIModelFunc(eligibleTables),
   374  	}
   375  	c.JSON(http.StatusOK, tables)
   376  }
   377  
   378  // updateChangefeed handles update changefeed request,
   379  // it returns the updated changefeedInfo
   380  // Can only update a changefeed's: TargetTs, SinkURI,
   381  // ReplicaConfig, PDAddrs, CAPath, CertPath, KeyPath,
   382  // SyncPointEnabled, SyncPointInterval
   383  // UpdateChangefeed updates a changefeed
   384  // @Summary Update a changefeed
   385  // @Description Update a changefeed
   386  // @Tags changefeed,v2
   387  // @Accept json
   388  // @Produce json
   389  // @Param changefeed_id  path  string  true  "changefeed_id"
   390  // @Param namespace query string false "default"
   391  // @Param changefeedConfig body ChangefeedConfig true "changefeed config"
   392  // @Success 200 {object} ChangeFeedInfo
   393  // @Failure 500,400 {object} model.HTTPError
   394  // @Router /api/v2/changefeeds/{changefeed_id} [put]
   395  func (h *OpenAPIV2) updateChangefeed(c *gin.Context) {
   396  	ctx := c.Request.Context()
   397  
   398  	namespace := getNamespaceValueWithDefault(c)
   399  	changefeedID := model.ChangeFeedID{Namespace: namespace, ID: c.Param(api.APIOpVarChangefeedID)}
   400  	if err := model.ValidateChangefeedID(changefeedID.ID); err != nil {
   401  		_ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s",
   402  			changefeedID.ID))
   403  		return
   404  	}
   405  
   406  	owner, err := h.capture.GetOwner()
   407  	if err != nil {
   408  		_ = c.Error(errors.Trace(err))
   409  		return
   410  	}
   411  
   412  	oldCfInfo, err := h.capture.StatusProvider().GetChangeFeedInfo(ctx, changefeedID)
   413  	if err != nil {
   414  		_ = c.Error(err)
   415  		return
   416  	}
   417  
   418  	switch oldCfInfo.State {
   419  	case model.StateStopped, model.StateFailed:
   420  	default:
   421  		_ = c.Error(
   422  			cerror.ErrChangefeedUpdateRefused.GenWithStackByArgs(
   423  				"can only update changefeed config when it is stopped or failed",
   424  			),
   425  		)
   426  		return
   427  	}
   428  
   429  	cfStatus, err := h.capture.StatusProvider().GetChangeFeedStatus(ctx, changefeedID)
   430  	if err != nil {
   431  		_ = c.Error(err)
   432  		return
   433  	}
   434  
   435  	oldCfInfo.Namespace = changefeedID.Namespace
   436  	oldCfInfo.ID = changefeedID.ID
   437  	OldUpInfo, err := h.capture.GetUpstreamInfo(ctx, oldCfInfo.UpstreamID,
   438  		oldCfInfo.Namespace)
   439  	if err != nil {
   440  		_ = c.Error(err)
   441  		return
   442  	}
   443  
   444  	updateCfConfig := &ChangefeedConfig{}
   445  	if err = c.BindJSON(updateCfConfig); err != nil {
   446  		_ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err))
   447  		return
   448  	}
   449  
   450  	if err = h.helpers.verifyUpstream(ctx, updateCfConfig, oldCfInfo); err != nil {
   451  		_ = c.Error(errors.Trace(err))
   452  		return
   453  	}
   454  
   455  	log.Info("Old ChangeFeed and Upstream Info",
   456  		zap.String("changefeedInfo", oldCfInfo.String()),
   457  		zap.Any("upstreamInfo", OldUpInfo))
   458  
   459  	upManager, err := h.capture.GetUpstreamManager()
   460  	if err != nil {
   461  		_ = c.Error(err)
   462  		return
   463  	}
   464  
   465  	var storage tidbkv.Storage
   466  	// if PDAddrs is not empty, use it to create a new kvstore
   467  	// Note: upManager is nil in some unit test cases
   468  	if len(updateCfConfig.PDAddrs) != 0 || upManager == nil {
   469  		pdAddrs := updateCfConfig.PDAddrs
   470  		credentials := updateCfConfig.PDConfig.toCredential()
   471  		storage, err = h.helpers.createTiStore(pdAddrs, credentials)
   472  		if err != nil {
   473  			_ = c.Error(errors.Trace(err))
   474  		}
   475  	} else { // get the upstream of the changefeed to get the kvstore
   476  		up, ok := upManager.Get(oldCfInfo.UpstreamID)
   477  		if !ok {
   478  			_ = c.Error(errors.New(fmt.Sprintf("upstream %d not found", oldCfInfo.UpstreamID)))
   479  			return
   480  		}
   481  		storage = up.KVStorage
   482  	}
   483  
   484  	newCfInfo, newUpInfo, err := h.helpers.verifyUpdateChangefeedConfig(ctx,
   485  		updateCfConfig, oldCfInfo, OldUpInfo, storage, cfStatus.CheckpointTs)
   486  	if err != nil {
   487  		_ = c.Error(errors.Trace(err))
   488  		return
   489  	}
   490  
   491  	log.Info("New ChangeFeed and Upstream Info",
   492  		zap.String("changefeedInfo", newCfInfo.String()),
   493  		zap.Any("upstreamInfo", newUpInfo))
   494  
   495  	err = owner.
   496  		UpdateChangefeedAndUpstream(ctx, newUpInfo, newCfInfo)
   497  	if err != nil {
   498  		_ = c.Error(errors.Trace(err))
   499  		return
   500  	}
   501  	c.JSON(http.StatusOK, toAPIModel(newCfInfo,
   502  		cfStatus.ResolvedTs, cfStatus.CheckpointTs, nil, true))
   503  }
   504  
   505  // getChangefeed get detailed info of a changefeed
   506  // @Summary Get changefeed
   507  // @Description get detail information of a changefeed
   508  // @Tags changefeed,v2
   509  // @Accept json
   510  // @Produce json
   511  // @Param changefeed_id  path  string  true  "changefeed_id"
   512  // @Param namespace query string false "default"
   513  // @Success 200 {object} ChangeFeedInfo
   514  // @Failure 500,400 {object} model.HTTPError
   515  // @Router /api/v2/changefeeds/{changefeed_id} [get]
   516  func (h *OpenAPIV2) getChangeFeed(c *gin.Context) {
   517  	ctx := c.Request.Context()
   518  	namespace := getNamespaceValueWithDefault(c)
   519  	changefeedID := model.ChangeFeedID{Namespace: namespace, ID: c.Param(api.APIOpVarChangefeedID)}
   520  	if err := model.ValidateChangefeedID(changefeedID.ID); err != nil {
   521  		_ = c.Error(
   522  			cerror.ErrAPIInvalidParam.GenWithStack(
   523  				"invalid changefeed_id: %s",
   524  				changefeedID.ID,
   525  			))
   526  		return
   527  	}
   528  	cfInfo, err := h.capture.StatusProvider().GetChangeFeedInfo(
   529  		ctx,
   530  		changefeedID,
   531  	)
   532  	if err != nil {
   533  		_ = c.Error(err)
   534  		return
   535  	}
   536  
   537  	status, err := h.capture.StatusProvider().GetChangeFeedStatus(
   538  		ctx,
   539  		changefeedID,
   540  	)
   541  	if err != nil {
   542  		_ = c.Error(err)
   543  		return
   544  	}
   545  
   546  	taskStatus := make([]model.CaptureTaskStatus, 0)
   547  	if cfInfo.State == model.StateNormal {
   548  		processorInfos, err := h.capture.StatusProvider().GetAllTaskStatuses(
   549  			ctx,
   550  			changefeedID,
   551  		)
   552  		if err != nil {
   553  			_ = c.Error(err)
   554  			return
   555  		}
   556  		for captureID, status := range processorInfos {
   557  			tables := make([]int64, 0)
   558  			for tableID := range status.Tables {
   559  				tables = append(tables, tableID)
   560  			}
   561  			taskStatus = append(taskStatus,
   562  				model.CaptureTaskStatus{
   563  					CaptureID: captureID, Tables: tables,
   564  					Operation: status.Operation,
   565  				})
   566  		}
   567  	}
   568  	detail := toAPIModel(cfInfo, status.ResolvedTs,
   569  		status.CheckpointTs, taskStatus, true)
   570  	c.JSON(http.StatusOK, detail)
   571  }
   572  
   573  // deleteChangefeed handles delete changefeed request
   574  // RemoveChangefeed removes a changefeed
   575  // @Summary Remove a changefeed
   576  // @Description Remove a changefeed
   577  // @Tags changefeed,v2
   578  // @Accept json
   579  // @Produce json
   580  // @Param changefeed_id path string true "changefeed_id"
   581  // @Param namespace query string false "default"
   582  // @Success 200 {object} EmptyResponse
   583  // @Failure 500,400 {object} model.HTTPError
   584  // @Router	/api/v2/changefeeds/{changefeed_id} [delete]
   585  func (h *OpenAPIV2) deleteChangefeed(c *gin.Context) {
   586  	ctx := c.Request.Context()
   587  	namespace := getNamespaceValueWithDefault(c)
   588  	changefeedID := model.ChangeFeedID{Namespace: namespace, ID: c.Param(api.APIOpVarChangefeedID)}
   589  	if err := model.ValidateChangefeedID(changefeedID.ID); err != nil {
   590  		_ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s",
   591  			changefeedID.ID))
   592  		return
   593  	}
   594  	ctrl, err := h.capture.GetController()
   595  	if err != nil {
   596  		_ = c.Error(err)
   597  		return
   598  	}
   599  	exist, err := ctrl.IsChangefeedExists(ctx, changefeedID)
   600  	if err != nil {
   601  		if cerror.ErrChangeFeedNotExists.Equal(err) {
   602  			c.JSON(http.StatusOK, &EmptyResponse{})
   603  			return
   604  		}
   605  		_ = c.Error(err)
   606  		return
   607  	}
   608  	if !exist {
   609  		c.JSON(http.StatusOK, &EmptyResponse{})
   610  		return
   611  	}
   612  
   613  	// todo: controller call metastroe api to remove the changefeed
   614  	job := model.AdminJob{
   615  		CfID: changefeedID,
   616  		Type: model.AdminRemove,
   617  	}
   618  
   619  	if err := api.HandleOwnerJob(ctx, h.capture, job); err != nil {
   620  		_ = c.Error(err)
   621  		return
   622  	}
   623  
   624  	// Owner needs at least two ticks to remove a changefeed,
   625  	// we need to wait for it.
   626  	err = retry.Do(ctx, func() error {
   627  		exist, err = ctrl.IsChangefeedExists(ctx, changefeedID)
   628  		if err != nil {
   629  			if strings.Contains(err.Error(), "ErrChangeFeedNotExists") {
   630  				return nil
   631  			}
   632  			return err
   633  		}
   634  		if !exist {
   635  			return nil
   636  		}
   637  		return cerror.ErrChangeFeedDeletionUnfinished.GenWithStackByArgs(changefeedID)
   638  	},
   639  		retry.WithMaxTries(100),         // max retry duration is 1 minute
   640  		retry.WithBackoffBaseDelay(600), // default owner tick interval is 200ms
   641  		retry.WithIsRetryableErr(cerror.IsRetryableError))
   642  
   643  	if err != nil {
   644  		_ = c.Error(err)
   645  		return
   646  	}
   647  	c.JSON(http.StatusOK, &EmptyResponse{})
   648  }
   649  
   650  // todo: remove this API
   651  // getChangeFeedMetaInfo returns the metaInfo of a changefeed
   652  func (h *OpenAPIV2) getChangeFeedMetaInfo(c *gin.Context) {
   653  	ctx := c.Request.Context()
   654  
   655  	namespace := getNamespaceValueWithDefault(c)
   656  	changefeedID := model.ChangeFeedID{Namespace: namespace, ID: c.Param(api.APIOpVarChangefeedID)}
   657  	if err := model.ValidateChangefeedID(changefeedID.ID); err != nil {
   658  		_ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s",
   659  			changefeedID.ID))
   660  		return
   661  	}
   662  	info, err := h.capture.StatusProvider().GetChangeFeedInfo(ctx, changefeedID)
   663  	if err != nil {
   664  		_ = c.Error(err)
   665  		return
   666  	}
   667  	status, err := h.capture.StatusProvider().GetChangeFeedStatus(
   668  		ctx,
   669  		changefeedID,
   670  	)
   671  	if err != nil {
   672  		_ = c.Error(err)
   673  		return
   674  	}
   675  	taskStatus := make([]model.CaptureTaskStatus, 0)
   676  	if info.State == model.StateNormal {
   677  		processorInfos, err := h.capture.StatusProvider().GetAllTaskStatuses(
   678  			ctx,
   679  			changefeedID,
   680  		)
   681  		if err != nil {
   682  			_ = c.Error(err)
   683  			return
   684  		}
   685  		for captureID, status := range processorInfos {
   686  			tables := make([]int64, 0)
   687  			for tableID := range status.Tables {
   688  				tables = append(tables, tableID)
   689  			}
   690  			taskStatus = append(taskStatus,
   691  				model.CaptureTaskStatus{
   692  					CaptureID: captureID, Tables: tables,
   693  					Operation: status.Operation,
   694  				})
   695  		}
   696  	}
   697  	c.JSON(http.StatusOK, toAPIModel(info, status.ResolvedTs, status.CheckpointTs,
   698  		taskStatus, false))
   699  }
   700  
   701  // resumeChangefeed handles resume changefeed request.
   702  // ResumeChangefeed resumes a changefeed
   703  // @Summary Resume a changefeed
   704  // @Description Resume a changefeed
   705  // @Tags changefeed,v2
   706  // @Accept json
   707  // @Produce json
   708  // @Param changefeed_id path string true "changefeed_id"
   709  // @Param namespace query string false "default"
   710  // @Param resumeConfig body ResumeChangefeedConfig true "resume config"
   711  // @Success 200 {object} EmptyResponse
   712  // @Failure 500,400 {object} model.HTTPError
   713  // @Router	/api/v2/changefeeds/{changefeed_id}/resume [post]
   714  func (h *OpenAPIV2) resumeChangefeed(c *gin.Context) {
   715  	ctx := c.Request.Context()
   716  	namespace := getNamespaceValueWithDefault(c)
   717  	changefeedID := model.ChangeFeedID{Namespace: namespace, ID: c.Param(api.APIOpVarChangefeedID)}
   718  	err := model.ValidateChangefeedID(changefeedID.ID)
   719  	if err != nil {
   720  		_ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s",
   721  			changefeedID.ID))
   722  		return
   723  	}
   724  
   725  	_, err = h.capture.StatusProvider().GetChangeFeedInfo(ctx, changefeedID)
   726  	if err != nil {
   727  		_ = c.Error(err)
   728  		return
   729  	}
   730  
   731  	cfg := new(ResumeChangefeedConfig)
   732  	if err := c.BindJSON(&cfg); err != nil {
   733  		_ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err))
   734  		return
   735  	}
   736  	status, err := h.capture.StatusProvider().GetChangeFeedStatus(ctx, changefeedID)
   737  	if err != nil {
   738  		_ = c.Error(err)
   739  		return
   740  	}
   741  
   742  	var pdClient pd.Client
   743  	// if PDAddrs is empty, use the default pdClient
   744  	if len(cfg.PDAddrs) == 0 {
   745  		up, err := getCaptureDefaultUpstream(h.capture)
   746  		if err != nil {
   747  			_ = c.Error(err)
   748  			return
   749  		}
   750  		pdClient = up.PDClient
   751  	} else {
   752  		credential := cfg.PDConfig.toCredential()
   753  		timeoutCtx, cancel := context.WithTimeout(ctx, 30*time.Second)
   754  		defer cancel()
   755  		pdClient, err = h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential)
   756  		if err != nil {
   757  			_ = c.Error(cerror.WrapError(cerror.ErrAPIInvalidParam, err))
   758  			return
   759  		}
   760  		defer pdClient.Close()
   761  	}
   762  	// If there is no overrideCheckpointTs, then check whether the currentCheckpointTs is smaller than gc safepoint or not.
   763  	newCheckpointTs := status.CheckpointTs
   764  	if cfg.OverwriteCheckpointTs != 0 {
   765  		newCheckpointTs = cfg.OverwriteCheckpointTs
   766  	}
   767  	if err := h.helpers.verifyResumeChangefeedConfig(
   768  		ctx,
   769  		pdClient,
   770  		h.capture.GetEtcdClient().GetEnsureGCServiceID(gc.EnsureGCServiceResuming),
   771  		changefeedID,
   772  		newCheckpointTs); err != nil {
   773  		_ = c.Error(err)
   774  		return
   775  	}
   776  	needRemoveGCSafePoint := false
   777  	defer func() {
   778  		if !needRemoveGCSafePoint {
   779  			return
   780  		}
   781  		err := gc.UndoEnsureChangefeedStartTsSafety(
   782  			ctx,
   783  			pdClient,
   784  			h.capture.GetEtcdClient().GetEnsureGCServiceID(gc.EnsureGCServiceResuming),
   785  			changefeedID,
   786  		)
   787  		if err != nil {
   788  			_ = c.Error(err)
   789  			return
   790  		}
   791  	}()
   792  
   793  	job := model.AdminJob{
   794  		CfID:                  changefeedID,
   795  		Type:                  model.AdminResume,
   796  		OverwriteCheckpointTs: cfg.OverwriteCheckpointTs,
   797  	}
   798  
   799  	if err := api.HandleOwnerJob(ctx, h.capture, job); err != nil {
   800  		needRemoveGCSafePoint = true
   801  		_ = c.Error(err)
   802  		return
   803  	}
   804  	c.JSON(http.StatusOK, &EmptyResponse{})
   805  }
   806  
   807  // pauseChangefeed handles pause changefeed request
   808  // PauseChangefeed pauses a changefeed
   809  // @Summary Pause a changefeed
   810  // @Description Pause a changefeed
   811  // @Tags changefeed,v2
   812  // @Accept json
   813  // @Produce json
   814  // @Param changefeed_id  path  string  true  "changefeed_id"
   815  // @Param namespace query string false "default"
   816  // @Success 200 {object} EmptyResponse
   817  // @Failure 500,400 {object} model.HTTPError
   818  // @Router /api/v2/changefeeds/{changefeed_id}/pause [post]
   819  func (h *OpenAPIV2) pauseChangefeed(c *gin.Context) {
   820  	ctx := c.Request.Context()
   821  
   822  	namespace := getNamespaceValueWithDefault(c)
   823  	changefeedID := model.ChangeFeedID{Namespace: namespace, ID: c.Param(api.APIOpVarChangefeedID)}
   824  	if err := model.ValidateChangefeedID(changefeedID.ID); err != nil {
   825  		_ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s",
   826  			changefeedID.ID))
   827  		return
   828  	}
   829  	// check if the changefeed exists
   830  	_, err := h.capture.StatusProvider().GetChangeFeedStatus(ctx, changefeedID)
   831  	if err != nil {
   832  		_ = c.Error(err)
   833  		return
   834  	}
   835  
   836  	job := model.AdminJob{
   837  		CfID: changefeedID,
   838  		Type: model.AdminStop,
   839  	}
   840  
   841  	if err := api.HandleOwnerJob(ctx, h.capture, job); err != nil {
   842  		_ = c.Error(err)
   843  		return
   844  	}
   845  	c.JSON(http.StatusOK, &EmptyResponse{})
   846  }
   847  
   848  func (h *OpenAPIV2) status(c *gin.Context) {
   849  	ctx := c.Request.Context()
   850  
   851  	namespace := getNamespaceValueWithDefault(c)
   852  	changefeedID := model.ChangeFeedID{Namespace: namespace, ID: c.Param(api.APIOpVarChangefeedID)}
   853  	if err := model.ValidateChangefeedID(changefeedID.ID); err != nil {
   854  		_ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s",
   855  			changefeedID.ID))
   856  		return
   857  	}
   858  	info, err := h.capture.StatusProvider().GetChangeFeedInfo(ctx, changefeedID)
   859  	if err != nil {
   860  		_ = c.Error(err)
   861  		return
   862  	}
   863  	status, err := h.capture.StatusProvider().GetChangeFeedStatus(
   864  		ctx,
   865  		changefeedID,
   866  	)
   867  	if err != nil {
   868  		_ = c.Error(err)
   869  		return
   870  	}
   871  	var lastError *RunningError
   872  	if info.Error != nil &&
   873  		oracle.GetTimeFromTS(status.CheckpointTs).Before(info.Error.Time) {
   874  		lastError = &RunningError{
   875  			Time:    &info.Error.Time,
   876  			Addr:    info.Error.Addr,
   877  			Code:    info.Error.Code,
   878  			Message: info.Error.Message,
   879  		}
   880  	}
   881  	var lastWarning *RunningError
   882  	if info.Warning != nil &&
   883  		oracle.GetTimeFromTS(status.CheckpointTs).Before(info.Warning.Time) {
   884  		lastWarning = &RunningError{
   885  			Time:    &info.Warning.Time,
   886  			Addr:    info.Warning.Addr,
   887  			Code:    info.Warning.Code,
   888  			Message: info.Warning.Message,
   889  		}
   890  	}
   891  
   892  	c.JSON(http.StatusOK, &ChangefeedStatus{
   893  		State:        string(info.State),
   894  		CheckpointTs: status.CheckpointTs,
   895  		ResolvedTs:   status.ResolvedTs,
   896  		LastError:    lastError,
   897  		LastWarning:  lastWarning,
   898  	})
   899  }
   900  
   901  // synced get the synced status of a changefeed
   902  // @Summary Get synced status
   903  // @Description get the synced status of a changefeed
   904  // @Tags changefeed,v2
   905  // @Accept json
   906  // @Produce json
   907  // @Param changefeed_id  path  string  true  "changefeed_id"
   908  // @Param namespace query string false "default"
   909  // @Success 200 {object} SyncedStatus
   910  // @Failure 500,400 {object} model.HTTPError
   911  // @Router /api/v2/changefeeds/{changefeed_id}/synced [get]
   912  func (h *OpenAPIV2) synced(c *gin.Context) {
   913  	ctx := c.Request.Context()
   914  
   915  	namespace := getNamespaceValueWithDefault(c)
   916  	changefeedID := model.ChangeFeedID{Namespace: namespace, ID: c.Param(api.APIOpVarChangefeedID)}
   917  	if err := model.ValidateChangefeedID(changefeedID.ID); err != nil {
   918  		_ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s",
   919  			changefeedID.ID))
   920  		return
   921  	}
   922  
   923  	status, err := h.capture.StatusProvider().GetChangeFeedSyncedStatus(ctx, changefeedID)
   924  	if err != nil {
   925  		_ = c.Error(err)
   926  		return
   927  	}
   928  
   929  	log.Info("Get changefeed synced status:", zap.Any("status", status), zap.Any("changefeedID", changefeedID))
   930  
   931  	cfg := &ChangefeedConfig{ReplicaConfig: GetDefaultReplicaConfig()}
   932  	if (status.SyncedCheckInterval != 0) && (status.CheckpointInterval != 0) {
   933  		cfg.ReplicaConfig.SyncedStatus.CheckpointInterval = status.CheckpointInterval
   934  		cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval = status.SyncedCheckInterval
   935  	}
   936  
   937  	// try to get pd client to get pd time, and determine synced status based on the pd time
   938  	if len(cfg.PDAddrs) == 0 {
   939  		up, err := getCaptureDefaultUpstream(h.capture)
   940  		if err != nil {
   941  			_ = c.Error(err)
   942  			return
   943  		}
   944  		cfg.PDConfig = getUpstreamPDConfig(up)
   945  	}
   946  	credential := cfg.PDConfig.toCredential()
   947  
   948  	timeoutCtx, cancel := context.WithTimeout(ctx, timeout)
   949  	defer cancel()
   950  
   951  	pdClient, err := h.helpers.getPDClient(timeoutCtx, cfg.PDAddrs, credential)
   952  	if err != nil {
   953  		// case 1. we can't get pd client, pd may be unavailable.
   954  		//         if pullerResolvedTs - checkpointTs > checkpointInterval, data is not synced
   955  		//         otherwise, if pd is unavailable, we decide data whether is synced based on
   956  		//         the time difference between current time and lastSyncedTs.
   957  		var message string
   958  		if (oracle.ExtractPhysical(status.PullerResolvedTs) - oracle.ExtractPhysical(status.CheckpointTs)) >
   959  			cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 {
   960  			message = fmt.Sprintf("%s. Besides the data is not finish syncing", err.Error())
   961  		} else {
   962  			message = fmt.Sprintf("%s. You should check the pd status first. If pd status is normal, means we don't finish sync data. "+
   963  				"If pd is offline, please check whether we satisfy the condition that "+
   964  				"the time difference from lastSyncedTs to the current time from the time zone of pd is greater than %v secs. "+
   965  				"If it's satisfied, means the data syncing is totally finished", err, cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval)
   966  		}
   967  		c.JSON(http.StatusOK, SyncedStatus{
   968  			Synced:           false,
   969  			SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)),
   970  			PullerResolvedTs: model.JSONTime(oracle.GetTimeFromTS(status.PullerResolvedTs)),
   971  			LastSyncedTs:     model.JSONTime(oracle.GetTimeFromTS(status.LastSyncedTs)),
   972  			NowTs:            model.JSONTime(time.Unix(0, 0)),
   973  			Info:             message,
   974  		})
   975  		return
   976  	}
   977  	defer pdClient.Close()
   978  	// get time from pd
   979  	physicalNow, _, _ := pdClient.GetTS(ctx)
   980  
   981  	// We can normally get pd time. Thus we determine synced status based on physicalNow, lastSyncedTs, checkpointTs and pullerResolvedTs
   982  	if (physicalNow-oracle.ExtractPhysical(status.LastSyncedTs) > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000) &&
   983  		(physicalNow-oracle.ExtractPhysical(status.CheckpointTs) < cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000) {
   984  		// case 2: If physcialNow - lastSyncedTs > SyncedCheckInterval && physcialNow - CheckpointTs < CheckpointInterval
   985  		//         --> reach strict synced status
   986  		c.JSON(http.StatusOK, SyncedStatus{
   987  			Synced:           true,
   988  			SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)),
   989  			PullerResolvedTs: model.JSONTime(oracle.GetTimeFromTS(status.PullerResolvedTs)),
   990  			LastSyncedTs:     model.JSONTime(oracle.GetTimeFromTS(status.LastSyncedTs)),
   991  			NowTs:            model.JSONTime(time.Unix(physicalNow/1e3, 0)),
   992  			Info:             "Data syncing is finished",
   993  		})
   994  		return
   995  	}
   996  
   997  	if physicalNow-oracle.ExtractPhysical(status.LastSyncedTs) > cfg.ReplicaConfig.SyncedStatus.SyncedCheckInterval*1000 {
   998  		// case 3: If physcialNow - lastSyncedTs > SyncedCheckInterval && physcialNow - CheckpointTs > CheckpointInterval
   999  		//         we should consider the situation that pd or tikv region is not healthy to block the advancing resolveTs.
  1000  		//         if pullerResolvedTs - checkpointTs > CheckpointInterval-->  data is not synced
  1001  		//         otherwise, if pd & tikv is healthy --> data is not synced
  1002  		//                    if not healthy --> data is synced
  1003  		var message string
  1004  		if (oracle.ExtractPhysical(status.PullerResolvedTs) - oracle.ExtractPhysical(status.CheckpointTs)) <
  1005  			cfg.ReplicaConfig.SyncedStatus.CheckpointInterval*1000 {
  1006  			message = fmt.Sprintf("Please check whether PD is online and TiKV Regions are all available. " +
  1007  				"If PD is offline or some TiKV regions are not available, it means that the data syncing process is complete. " +
  1008  				"To check whether TiKV regions are all available, you can view " +
  1009  				"'TiKV-Details' > 'Resolved-Ts' > 'Max Leader Resolved TS gap' on Grafana. " +
  1010  				"If the gap is large, such as a few minutes, it means that some regions in TiKV are unavailable. " +
  1011  				"Otherwise, if the gap is small and PD is online, it means the data syncing is incomplete, so please wait")
  1012  		} else {
  1013  			message = "The data syncing is not finished, please wait"
  1014  		}
  1015  		c.JSON(http.StatusOK, SyncedStatus{
  1016  			Synced:           false,
  1017  			SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)),
  1018  			PullerResolvedTs: model.JSONTime(oracle.GetTimeFromTS(status.PullerResolvedTs)),
  1019  			LastSyncedTs:     model.JSONTime(oracle.GetTimeFromTS(status.LastSyncedTs)),
  1020  			NowTs:            model.JSONTime(time.Unix(physicalNow/1e3, 0)),
  1021  			Info:             message,
  1022  		})
  1023  		return
  1024  	}
  1025  
  1026  	// case	4: If physcialNow - lastSyncedTs < SyncedCheckInterval --> data is not synced
  1027  	c.JSON(http.StatusOK, SyncedStatus{
  1028  		Synced:           false,
  1029  		SinkCheckpointTs: model.JSONTime(oracle.GetTimeFromTS(status.CheckpointTs)),
  1030  		PullerResolvedTs: model.JSONTime(oracle.GetTimeFromTS(status.PullerResolvedTs)),
  1031  		LastSyncedTs:     model.JSONTime(oracle.GetTimeFromTS(status.LastSyncedTs)),
  1032  		NowTs:            model.JSONTime(time.Unix(physicalNow/1e3, 0)),
  1033  		Info:             "The data syncing is not finished, please wait",
  1034  	})
  1035  }
  1036  
  1037  func toAPIModel(
  1038  	info *model.ChangeFeedInfo,
  1039  	resolvedTs uint64,
  1040  	checkpointTs uint64,
  1041  	taskStatus []model.CaptureTaskStatus,
  1042  	maskSinkURI bool,
  1043  ) *ChangeFeedInfo {
  1044  	var runningError *RunningError
  1045  
  1046  	// if the state is normal, we shall not return the error info
  1047  	// because changefeed will is retrying. errors will confuse the users
  1048  	if info.State != model.StateNormal && info.Error != nil {
  1049  		runningError = &RunningError{
  1050  			Addr:    info.Error.Addr,
  1051  			Code:    info.Error.Code,
  1052  			Message: info.Error.Message,
  1053  		}
  1054  	}
  1055  
  1056  	sinkURI := info.SinkURI
  1057  	var err error
  1058  	if maskSinkURI {
  1059  		sinkURI, err = util.MaskSinkURI(sinkURI)
  1060  		if err != nil {
  1061  			log.Error("failed to mask sink URI", zap.Error(err))
  1062  		}
  1063  	}
  1064  
  1065  	apiInfoModel := &ChangeFeedInfo{
  1066  		UpstreamID:     info.UpstreamID,
  1067  		Namespace:      info.Namespace,
  1068  		ID:             info.ID,
  1069  		SinkURI:        sinkURI,
  1070  		CreateTime:     info.CreateTime,
  1071  		StartTs:        info.StartTs,
  1072  		TargetTs:       info.TargetTs,
  1073  		AdminJobType:   info.AdminJobType,
  1074  		Config:         ToAPIReplicaConfig(info.Config),
  1075  		State:          info.State,
  1076  		Error:          runningError,
  1077  		CreatorVersion: info.CreatorVersion,
  1078  		CheckpointTs:   checkpointTs,
  1079  		ResolvedTs:     resolvedTs,
  1080  		CheckpointTime: model.JSONTime(oracle.GetTimeFromTS(checkpointTs)),
  1081  		TaskStatus:     taskStatus,
  1082  	}
  1083  	return apiInfoModel
  1084  }
  1085  
  1086  func getCaptureDefaultUpstream(cp capture.Capture) (*upstream.Upstream, error) {
  1087  	upManager, err := cp.GetUpstreamManager()
  1088  	if err != nil {
  1089  		return nil, errors.Trace(err)
  1090  	}
  1091  	up, err := upManager.GetDefaultUpstream()
  1092  	if err != nil {
  1093  		return nil, errors.Trace(err)
  1094  	}
  1095  	return up, nil
  1096  }
  1097  
  1098  func getUpstreamPDConfig(up *upstream.Upstream) PDConfig {
  1099  	return PDConfig{
  1100  		PDAddrs:  up.PdEndpoints,
  1101  		KeyPath:  up.SecurityConfig.KeyPath,
  1102  		CAPath:   up.SecurityConfig.CAPath,
  1103  		CertPath: up.SecurityConfig.CertPath,
  1104  	}
  1105  }