github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cmd/client_changefeed.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 cmd
    15  
    16  import (
    17  	"context"
    18  	"encoding/json"
    19  	"fmt"
    20  	"net/url"
    21  	"strings"
    22  	"time"
    23  
    24  	"github.com/fatih/color"
    25  	"github.com/google/uuid"
    26  	"github.com/pingcap/errors"
    27  	"github.com/pingcap/log"
    28  	"github.com/pingcap/ticdc/cdc"
    29  	"github.com/pingcap/ticdc/cdc/model"
    30  	"github.com/pingcap/ticdc/pkg/config"
    31  	"github.com/pingcap/ticdc/pkg/cyclic"
    32  	"github.com/pingcap/ticdc/pkg/cyclic/mark"
    33  	cerror "github.com/pingcap/ticdc/pkg/errors"
    34  	"github.com/pingcap/ticdc/pkg/security"
    35  	"github.com/pingcap/ticdc/pkg/util"
    36  	"github.com/pingcap/ticdc/pkg/version"
    37  	"github.com/pingcap/tidb/store/tikv/oracle"
    38  	"github.com/r3labs/diff"
    39  	"github.com/spf13/cobra"
    40  	"github.com/spf13/pflag"
    41  	"go.uber.org/zap"
    42  )
    43  
    44  var forceEnableOldValueProtocols = []string{
    45  	"canal",
    46  	"maxwell",
    47  }
    48  
    49  func newChangefeedCommand() *cobra.Command {
    50  	command := &cobra.Command{
    51  		Use:   "changefeed",
    52  		Short: "Manage changefeed (changefeed is a replication task)",
    53  	}
    54  	command.AddCommand(
    55  		newListChangefeedCommand(),
    56  		newQueryChangefeedCommand(),
    57  		newCreateChangefeedCommand(),
    58  		newUpdateChangefeedCommand(),
    59  		newStatisticsChangefeedCommand(),
    60  		newCreateChangefeedCyclicCommand(),
    61  	)
    62  	// Add pause, resume, remove changefeed
    63  	for _, cmd := range newAdminChangefeedCommand() {
    64  		command.AddCommand(cmd)
    65  	}
    66  	return command
    67  }
    68  
    69  func resumeChangefeedCheck(ctx context.Context, cmd *cobra.Command) error {
    70  	resp, err := applyOwnerChangefeedQuery(ctx, changefeedID, getCredential())
    71  	if err != nil {
    72  		return err
    73  	}
    74  	info := &cdc.ChangefeedResp{}
    75  	err = json.Unmarshal([]byte(resp), info)
    76  	if err != nil {
    77  		return err
    78  	}
    79  	return confirmLargeDataGap(ctx, cmd, info.TSO)
    80  }
    81  
    82  func newAdminChangefeedCommand() []*cobra.Command {
    83  	cmds := []*cobra.Command{
    84  		{
    85  			Use:   "pause",
    86  			Short: "Pause a replication task (changefeed)",
    87  			RunE: func(cmd *cobra.Command, args []string) error {
    88  				ctx := defaultContext
    89  				job := model.AdminJob{
    90  					CfID: changefeedID,
    91  					Type: model.AdminStop,
    92  				}
    93  				return applyAdminChangefeed(ctx, job, getCredential())
    94  			},
    95  		},
    96  		{
    97  			Use:   "resume",
    98  			Short: "Resume a paused replication task (changefeed)",
    99  			RunE: func(cmd *cobra.Command, args []string) error {
   100  				ctx := defaultContext
   101  				job := model.AdminJob{
   102  					CfID: changefeedID,
   103  					Type: model.AdminResume,
   104  				}
   105  				if err := resumeChangefeedCheck(ctx, cmd); err != nil {
   106  					return err
   107  				}
   108  				return applyAdminChangefeed(ctx, job, getCredential())
   109  			},
   110  		},
   111  		{
   112  			Use:   "remove",
   113  			Short: "Remove a replication task (changefeed)",
   114  			RunE: func(cmd *cobra.Command, args []string) error {
   115  				ctx := defaultContext
   116  				job := model.AdminJob{
   117  					CfID: changefeedID,
   118  					Type: model.AdminRemove,
   119  					Opts: &model.AdminJobOption{
   120  						ForceRemove: optForceRemove,
   121  					},
   122  				}
   123  				return applyAdminChangefeed(ctx, job, getCredential())
   124  			},
   125  		},
   126  	}
   127  
   128  	for _, cmd := range cmds {
   129  		cmd.PersistentFlags().StringVarP(&changefeedID, "changefeed-id", "c", "", "Replication task (changefeed) ID")
   130  		_ = cmd.MarkPersistentFlagRequired("changefeed-id")
   131  		if cmd.Use == "remove" {
   132  			cmd.PersistentFlags().BoolVarP(&optForceRemove, "force", "f", false, "remove all information of the changefeed")
   133  		}
   134  		if cmd.Use == "resume" {
   135  			cmd.PersistentFlags().BoolVar(&noConfirm, "no-confirm", false, "Don't ask user whether to ignore ineligible table")
   136  		}
   137  	}
   138  	return cmds
   139  }
   140  
   141  func newListChangefeedCommand() *cobra.Command {
   142  	command := &cobra.Command{
   143  		Use:   "list",
   144  		Short: "List all replication tasks (changefeeds) in TiCDC cluster",
   145  		RunE: func(cmd *cobra.Command, args []string) error {
   146  			ctx := defaultContext
   147  			_, raw, err := cdcEtcdCli.GetChangeFeeds(ctx)
   148  			if err != nil {
   149  				return err
   150  			}
   151  			changefeedIDs := make(map[string]struct{}, len(raw))
   152  			for id := range raw {
   153  				changefeedIDs[id] = struct{}{}
   154  			}
   155  			if changefeedListAll {
   156  				statuses, err := cdcEtcdCli.GetAllChangeFeedStatus(ctx)
   157  				if err != nil {
   158  					return err
   159  				}
   160  				for cid := range statuses {
   161  					changefeedIDs[cid] = struct{}{}
   162  				}
   163  			}
   164  			cfs := make([]*changefeedCommonInfo, 0, len(changefeedIDs))
   165  			for id := range changefeedIDs {
   166  				cfci := &changefeedCommonInfo{ID: id}
   167  				resp, err := applyOwnerChangefeedQuery(ctx, id, getCredential())
   168  				if err != nil {
   169  					// if no capture is available, the query will fail, just add a warning here
   170  					log.Warn("query changefeed info failed", zap.String("error", err.Error()))
   171  				} else {
   172  					info := &cdc.ChangefeedResp{}
   173  					err = json.Unmarshal([]byte(resp), info)
   174  					if err != nil {
   175  						return err
   176  					}
   177  					cfci.Summary = info
   178  				}
   179  				cfs = append(cfs, cfci)
   180  			}
   181  			return jsonPrint(cmd, cfs)
   182  		},
   183  	}
   184  	command.PersistentFlags().BoolVarP(&changefeedListAll, "all", "a", false, "List all replication tasks(including removed and finished)")
   185  	return command
   186  }
   187  
   188  func newQueryChangefeedCommand() *cobra.Command {
   189  	command := &cobra.Command{
   190  		Use:   "query",
   191  		Short: "Query information and status of a replication task (changefeed)",
   192  		RunE: func(cmd *cobra.Command, args []string) error {
   193  			ctx := defaultContext
   194  
   195  			if simplified {
   196  				resp, err := applyOwnerChangefeedQuery(ctx, changefeedID, getCredential())
   197  				if err != nil {
   198  					return err
   199  				}
   200  				cmd.Println(resp)
   201  				return nil
   202  			}
   203  
   204  			info, err := cdcEtcdCli.GetChangeFeedInfo(ctx, changefeedID)
   205  			if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) {
   206  				return err
   207  			}
   208  			status, _, err := cdcEtcdCli.GetChangeFeedStatus(ctx, changefeedID)
   209  			if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) {
   210  				return err
   211  			}
   212  			if err != nil && cerror.ErrChangeFeedNotExists.Equal(err) {
   213  				log.Error("This changefeed does not exist", zap.String("changefeed", changefeedID))
   214  				return err
   215  			}
   216  			taskPositions, err := cdcEtcdCli.GetAllTaskPositions(ctx, changefeedID)
   217  			if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) {
   218  				return err
   219  			}
   220  			var count uint64
   221  			for _, pinfo := range taskPositions {
   222  				count += pinfo.Count
   223  			}
   224  			processorInfos, err := cdcEtcdCli.GetAllTaskStatus(ctx, changefeedID)
   225  			if err != nil {
   226  				return err
   227  			}
   228  			taskStatus := make([]captureTaskStatus, 0, len(processorInfos))
   229  			for captureID, status := range processorInfos {
   230  				taskStatus = append(taskStatus, captureTaskStatus{CaptureID: captureID, TaskStatus: status})
   231  			}
   232  			meta := &cfMeta{Info: info, Status: status, Count: count, TaskStatus: taskStatus}
   233  			if info == nil {
   234  				log.Warn("This changefeed has been deleted, the residual meta data will be completely deleted within 24 hours.", zap.String("changgefeed", changefeedID))
   235  			}
   236  			return jsonPrint(cmd, meta)
   237  		},
   238  	}
   239  	command.PersistentFlags().BoolVarP(&simplified, "simple", "s", false, "Output simplified replication status")
   240  	command.PersistentFlags().StringVarP(&changefeedID, "changefeed-id", "c", "", "Replication task (changefeed) ID")
   241  	_ = command.MarkPersistentFlagRequired("changefeed-id")
   242  	return command
   243  }
   244  
   245  func verifyChangefeedParameters(ctx context.Context, cmd *cobra.Command, isCreate bool, credential *security.Credential, captureInfos []*model.CaptureInfo) (*model.ChangeFeedInfo, error) {
   246  	if isCreate {
   247  		if sinkURI == "" {
   248  			return nil, errors.New("Creating changefeed without a sink-uri")
   249  		}
   250  		if startTs == 0 {
   251  			ts, logical, err := pdCli.GetTS(ctx)
   252  			if err != nil {
   253  				return nil, err
   254  			}
   255  			startTs = oracle.ComposeTS(ts, logical)
   256  		}
   257  		if err := verifyStartTs(ctx, changefeedID, startTs); err != nil {
   258  			return nil, err
   259  		}
   260  		if err := confirmLargeDataGap(ctx, cmd, startTs); err != nil {
   261  			return nil, err
   262  		}
   263  		if err := verifyTargetTs(startTs, targetTs); err != nil {
   264  			return nil, err
   265  		}
   266  	}
   267  	cdcClusterVer, err := version.GetTiCDCClusterVersion(captureInfos)
   268  	if err != nil {
   269  		return nil, errors.Trace(err)
   270  	}
   271  	cfg := config.GetDefaultReplicaConfig()
   272  
   273  	if !cdcClusterVer.ShouldEnableOldValueByDefault() {
   274  		cfg.EnableOldValue = false
   275  		log.Warn("The TiCDC cluster is built from an older version, disabling old value by default.",
   276  			zap.String("version", cdcClusterVer.String()))
   277  	}
   278  
   279  	sortEngineFlag := cmd.Flag("sort-engine")
   280  	if !sortEngineFlag.Changed && !cdcClusterVer.ShouldEnableUnifiedSorterByDefault() {
   281  		sortEngine = model.SortInMemory
   282  		log.Warn("The TiCDC cluster is built from an older version, disabling Unified Sorter by default",
   283  			zap.String("version", cdcClusterVer.String()))
   284  	}
   285  
   286  	if len(configFile) > 0 {
   287  		if err := verifyReplicaConfig(configFile, "TiCDC changefeed", cfg); err != nil {
   288  			return nil, err
   289  		}
   290  	}
   291  	if disableGCSafePointCheck {
   292  		cfg.CheckGCSafePoint = false
   293  	}
   294  	if cyclicReplicaID != 0 || len(cyclicFilterReplicaIDs) != 0 {
   295  		if !(cyclicReplicaID != 0 && len(cyclicFilterReplicaIDs) != 0) {
   296  			return nil, errors.New("invalid cyclic config, please make sure using " +
   297  				"nonzero replica ID and specify filter replica IDs")
   298  		}
   299  		filter := make([]uint64, 0, len(cyclicFilterReplicaIDs))
   300  		for _, id := range cyclicFilterReplicaIDs {
   301  			filter = append(filter, uint64(id))
   302  		}
   303  		cfg.Cyclic = &config.CyclicConfig{
   304  			Enable:          true,
   305  			ReplicaID:       cyclicReplicaID,
   306  			FilterReplicaID: filter,
   307  			SyncDDL:         cyclicSyncDDL,
   308  			// TODO(neil) enable ID bucket.
   309  		}
   310  	}
   311  
   312  	if !cfg.EnableOldValue {
   313  		sinkURIParsed, err := url.Parse(sinkURI)
   314  		if err != nil {
   315  			return nil, cerror.WrapError(cerror.ErrSinkURIInvalid, err)
   316  		}
   317  
   318  		protocol := sinkURIParsed.Query().Get("protocol")
   319  		for _, fp := range forceEnableOldValueProtocols {
   320  			if protocol == fp {
   321  				log.Warn("Attempting to replicate without old value enabled. CDC will enable old value and continue.", zap.String("protocol", protocol))
   322  				cfg.EnableOldValue = true
   323  				break
   324  			}
   325  		}
   326  
   327  		if cfg.ForceReplicate {
   328  			log.Error("if use force replicate, old value feature must be enabled")
   329  			return nil, cerror.ErrOldValueNotEnabled.GenWithStackByArgs()
   330  		}
   331  	}
   332  
   333  	for _, rules := range cfg.Sink.DispatchRules {
   334  		switch strings.ToLower(rules.Dispatcher) {
   335  		case "rowid", "index-value":
   336  			if cfg.EnableOldValue {
   337  				cmd.Printf("[WARN] This index-value distribution mode "+
   338  					"does not guarantee row-level orderliness when "+
   339  					"switching on the old value, so please use caution! dispatch-rules: %#v", rules)
   340  			}
   341  		}
   342  	}
   343  	switch sortEngine {
   344  	case model.SortUnified, model.SortInMemory:
   345  	case model.SortInFile:
   346  		// obsolete. But we keep silent here. We create a Unified Sorter when the owner/processor sees this option
   347  		// for backward-compatibility.
   348  	default:
   349  		return nil, errors.Errorf("Creating changefeed with an invalid sort engine(%s), "+
   350  			"`%s` and `%s` are the only valid options.", sortEngine, model.SortUnified, model.SortInMemory)
   351  	}
   352  	info := &model.ChangeFeedInfo{
   353  		SinkURI:           sinkURI,
   354  		Opts:              make(map[string]string),
   355  		CreateTime:        time.Now(),
   356  		StartTs:           startTs,
   357  		TargetTs:          targetTs,
   358  		Config:            cfg,
   359  		Engine:            sortEngine,
   360  		State:             model.StateNormal,
   361  		SyncPointEnabled:  syncPointEnabled,
   362  		SyncPointInterval: syncPointInterval,
   363  		CreatorVersion:    version.ReleaseVersion,
   364  	}
   365  
   366  	// user is not allowed to set sort-dir at changefeed level
   367  	if sortDir != "" {
   368  		cmd.Printf(color.HiYellowString("[WARN] --sort-dir is deprecated in changefeed settings. " +
   369  			"Please use `cdc server --data-dir` to start the cdc server if possible, sort-dir will be set automatically. " +
   370  			"The --sort-dir here will be no-op\n"))
   371  		return nil, errors.New("Creating changefeed with `--sort-dir`, it's invalid")
   372  	}
   373  
   374  	if info.Engine == model.SortInFile {
   375  		cmd.Printf("[WARN] file sorter is obsolete. Unified Sorter is recommended. " +
   376  			"Adjust \"sort-engine\" to make use of the right sorter.\n" +
   377  			"A newer cluster will use Unified Sorter.\n")
   378  	}
   379  
   380  	tz, err := util.GetTimezone(timezone)
   381  	if err != nil {
   382  		return nil, errors.Annotate(err, "can not load timezone, Please specify the time zone through environment variable `TZ` or command line parameters `--tz`")
   383  	}
   384  
   385  	if isCreate {
   386  		ctx = util.PutTimezoneInCtx(ctx, tz)
   387  		ineligibleTables, eligibleTables, err := verifyTables(credential, cfg, startTs)
   388  		if err != nil {
   389  			return nil, err
   390  		}
   391  		if len(ineligibleTables) != 0 {
   392  			if cfg.ForceReplicate {
   393  				cmd.Printf("[WARN] force to replicate some ineligible tables, %#v\n", ineligibleTables)
   394  			} else {
   395  				cmd.Printf("[WARN] some tables are not eligible to replicate, %#v\n", ineligibleTables)
   396  				if !noConfirm {
   397  					cmd.Printf("Could you agree to ignore those tables, and continue to replicate [Y/N]\n")
   398  					var yOrN string
   399  					_, err := fmt.Scan(&yOrN)
   400  					if err != nil {
   401  						return nil, err
   402  					}
   403  					if strings.ToLower(strings.TrimSpace(yOrN)) != "y" {
   404  						cmd.Printf("No changefeed is created because you don't want to ignore some tables.\n")
   405  						return nil, nil
   406  					}
   407  				}
   408  			}
   409  		}
   410  		if cfg.Cyclic.IsEnabled() && !cyclic.IsTablesPaired(eligibleTables) {
   411  			return nil, errors.New("normal tables and mark tables are not paired, " +
   412  				"please run `cdc cli changefeed cyclic create-marktables`")
   413  		}
   414  	}
   415  
   416  	for _, opt := range opts {
   417  		s := strings.SplitN(opt, "=", 2)
   418  		if len(s) <= 0 {
   419  			cmd.Printf("omit opt: %s", opt)
   420  			continue
   421  		}
   422  
   423  		var key string
   424  		var value string
   425  
   426  		key = s[0]
   427  		if len(s) > 1 {
   428  			value = s[1]
   429  		}
   430  		info.Opts[key] = value
   431  	}
   432  
   433  	err = verifySink(ctx, info.SinkURI, info.Config, info.Opts)
   434  	if err != nil {
   435  		return nil, err
   436  	}
   437  	return info, nil
   438  }
   439  
   440  func changefeedConfigVariables(command *cobra.Command) {
   441  	command.PersistentFlags().Uint64Var(&startTs, "start-ts", 0, "Start ts of changefeed")
   442  	command.PersistentFlags().Uint64Var(&targetTs, "target-ts", 0, "Target ts of changefeed")
   443  	command.PersistentFlags().StringVar(&sinkURI, "sink-uri", "", "sink uri")
   444  	command.PersistentFlags().StringVar(&configFile, "config", "", "Path of the configuration file")
   445  	command.PersistentFlags().StringSliceVar(&opts, "opts", nil, "Extra options, in the `key=value` format")
   446  	command.PersistentFlags().StringVar(&sortEngine, "sort-engine", model.SortUnified, "sort engine used for data sort")
   447  	command.PersistentFlags().StringVar(&sortDir, "sort-dir", "", "directory used for data sort")
   448  	command.PersistentFlags().StringVar(&timezone, "tz", "SYSTEM", "timezone used when checking sink uri (changefeed timezone is determined by cdc server)")
   449  	command.PersistentFlags().Uint64Var(&cyclicReplicaID, "cyclic-replica-id", 0, "(Experimental) Cyclic replication replica ID of changefeed")
   450  	command.PersistentFlags().UintSliceVar(&cyclicFilterReplicaIDs, "cyclic-filter-replica-ids", []uint{}, "(Experimental) Cyclic replication filter replica ID of changefeed")
   451  	command.PersistentFlags().BoolVar(&cyclicSyncDDL, "cyclic-sync-ddl", true, "(Experimental) Cyclic replication sync DDL of changefeed")
   452  	command.PersistentFlags().BoolVar(&syncPointEnabled, "sync-point", false, "(Experimental) Set and Record syncpoint in replication(default off)")
   453  	command.PersistentFlags().DurationVar(&syncPointInterval, "sync-interval", 10*time.Minute, "(Experimental) Set the interval for syncpoint in replication(default 10min)")
   454  	_ = command.PersistentFlags().MarkHidden("sort-dir") //nolint:errcheck
   455  }
   456  
   457  func newCreateChangefeedCommand() *cobra.Command {
   458  	command := &cobra.Command{
   459  		Use:   "create",
   460  		Short: "Create a new replication task (changefeed)",
   461  		Long:  ``,
   462  		RunE: func(cmd *cobra.Command, args []string) error {
   463  			ctx := defaultContext
   464  			id := changefeedID
   465  			if id == "" {
   466  				id = uuid.New().String()
   467  			}
   468  			// validate the changefeedID first
   469  			if err := model.ValidateChangefeedID(id); err != nil {
   470  				return err
   471  			}
   472  
   473  			_, captureInfos, err := cdcEtcdCli.GetCaptures(ctx)
   474  			if err != nil {
   475  				return err
   476  			}
   477  			info, err := verifyChangefeedParameters(ctx, cmd, true /* isCreate */, getCredential(), captureInfos)
   478  			if err != nil {
   479  				return err
   480  			}
   481  			if info == nil {
   482  				return nil
   483  			}
   484  
   485  			infoStr, err := info.Marshal()
   486  			if err != nil {
   487  				return err
   488  			}
   489  			err = cdcEtcdCli.CreateChangefeedInfo(ctx, info, id)
   490  			if err != nil {
   491  				return err
   492  			}
   493  			cmd.Printf("Create changefeed successfully!\nID: %s\nInfo: %s\n", id, infoStr)
   494  			return nil
   495  		},
   496  	}
   497  	changefeedConfigVariables(command)
   498  	command.PersistentFlags().BoolVar(&noConfirm, "no-confirm", false, "Don't ask user whether to ignore ineligible table")
   499  	command.PersistentFlags().StringVarP(&changefeedID, "changefeed-id", "c", "", "Replication task (changefeed) ID")
   500  	command.PersistentFlags().BoolVarP(&disableGCSafePointCheck, "disable-gc-check", "", false, "Disable GC safe point check")
   501  
   502  	return command
   503  }
   504  
   505  func newUpdateChangefeedCommand() *cobra.Command {
   506  	command := &cobra.Command{
   507  		Use:   "update",
   508  		Short: "Update config of an existing replication task (changefeed)",
   509  		Long:  ``,
   510  		RunE: func(cmd *cobra.Command, args []string) (err error) {
   511  			ctx := defaultContext
   512  
   513  			old, err := cdcEtcdCli.GetChangeFeedInfo(ctx, changefeedID)
   514  			if err != nil {
   515  				return err
   516  			}
   517  			info, err := old.Clone()
   518  			if err != nil {
   519  				return err
   520  			}
   521  
   522  			cmd.Flags().Visit(func(flag *pflag.Flag) {
   523  				switch flag.Name {
   524  				case "target-ts":
   525  					info.TargetTs = targetTs
   526  				case "sink-uri":
   527  					info.SinkURI = sinkURI
   528  				case "config":
   529  					cfg := info.Config
   530  					if err = verifyReplicaConfig(configFile, "TiCDC changefeed", cfg); err != nil {
   531  						log.Error("decode config file error", zap.Error(err))
   532  					}
   533  				case "opts":
   534  					for _, opt := range opts {
   535  						s := strings.SplitN(opt, "=", 2)
   536  						if len(s) <= 0 {
   537  							cmd.Printf("omit opt: %s", opt)
   538  							continue
   539  						}
   540  
   541  						var key string
   542  						var value string
   543  						key = s[0]
   544  						if len(s) > 1 {
   545  							value = s[1]
   546  						}
   547  						info.Opts[key] = value
   548  					}
   549  
   550  				case "sort-engine":
   551  					info.Engine = sortEngine
   552  				case "cyclic-replica-id":
   553  					filter := make([]uint64, 0, len(cyclicFilterReplicaIDs))
   554  					for _, id := range cyclicFilterReplicaIDs {
   555  						filter = append(filter, uint64(id))
   556  					}
   557  					info.Config.Cyclic.FilterReplicaID = filter
   558  				case "cyclic-sync-ddl":
   559  					info.Config.Cyclic.SyncDDL = cyclicSyncDDL
   560  				case "sync-point":
   561  					info.SyncPointEnabled = syncPointEnabled
   562  				case "sync-interval":
   563  					info.SyncPointInterval = syncPointInterval
   564  				case "pd", "tz", "start-ts", "changefeed-id", "no-confirm":
   565  					// do nothing
   566  				default:
   567  					// use this default branch to prevent new added parameter is not added
   568  					log.Warn("unsupported flag, please report a bug", zap.String("flagName", flag.Name))
   569  				}
   570  			})
   571  			if err != nil {
   572  				return err
   573  			}
   574  
   575  			resp, err := applyOwnerChangefeedQuery(ctx, changefeedID, getCredential())
   576  			// if no cdc owner exists, allow user to update changefeed config
   577  			if err != nil && errors.Cause(err) != errOwnerNotFound {
   578  				return err
   579  			}
   580  			// Note that the correctness of the logic here depends on the return value of `/capture/owner/changefeed/query` interface.
   581  			// TODO: Using error codes instead of string containing judgments
   582  			if err == nil && !strings.Contains(resp, `"state": "stopped"`) {
   583  				return errors.Errorf("can only update changefeed config when it is stopped\nstatus: %s", resp)
   584  			}
   585  
   586  			changelog, err := diff.Diff(old, info)
   587  			if err != nil {
   588  				return err
   589  			}
   590  			if len(changelog) == 0 {
   591  				cmd.Printf("changefeed config is the same with the old one, do nothing\n")
   592  				return nil
   593  			}
   594  			cmd.Printf("Diff of changefeed config:\n")
   595  			for _, change := range changelog {
   596  				cmd.Printf("%+v\n", change)
   597  			}
   598  
   599  			if !noConfirm {
   600  				cmd.Printf("Could you agree to apply changes above to changefeed [Y/N]\n")
   601  				var yOrN string
   602  				_, err = fmt.Scan(&yOrN)
   603  				if err != nil {
   604  					return err
   605  				}
   606  				if strings.ToLower(strings.TrimSpace(yOrN)) != "y" {
   607  					cmd.Printf("No update to changefeed.\n")
   608  					return nil
   609  				}
   610  			}
   611  
   612  			err = cdcEtcdCli.SaveChangeFeedInfo(ctx, info, changefeedID)
   613  			if err != nil {
   614  				return err
   615  			}
   616  			infoStr, err := info.Marshal()
   617  			if err != nil {
   618  				return err
   619  			}
   620  			cmd.Printf("Update changefeed config successfully! "+
   621  				"Will take effect only if the changefeed has been paused before this command"+
   622  				"\nID: %s\nInfo: %s\n", changefeedID, infoStr)
   623  			return nil
   624  		},
   625  	}
   626  	changefeedConfigVariables(command)
   627  	command.PersistentFlags().StringVarP(&changefeedID, "changefeed-id", "c", "", "Replication task (changefeed) ID")
   628  	command.PersistentFlags().BoolVar(&noConfirm, "no-confirm", false, "Don't ask user whether to confirm update changefeed config")
   629  	_ = command.MarkPersistentFlagRequired("changefeed-id")
   630  
   631  	return command
   632  }
   633  
   634  func newStatisticsChangefeedCommand() *cobra.Command {
   635  	command := &cobra.Command{
   636  		Use:   "statistics",
   637  		Short: "Periodically check and output the status of a replicaiton task (changefeed)",
   638  		RunE: func(cmd *cobra.Command, args []string) error {
   639  			ctx := defaultContext
   640  			tick := time.NewTicker(time.Duration(interval) * time.Second)
   641  			lastTime := time.Now()
   642  			var lastCount uint64
   643  			for {
   644  				select {
   645  				case <-ctx.Done():
   646  					if err := ctx.Err(); err != nil {
   647  						return err
   648  					}
   649  				case <-tick.C:
   650  					now := time.Now()
   651  					status, _, err := cdcEtcdCli.GetChangeFeedStatus(ctx, changefeedID)
   652  					if err != nil {
   653  						return err
   654  					}
   655  					taskPositions, err := cdcEtcdCli.GetAllTaskPositions(ctx, changefeedID)
   656  					if err != nil {
   657  						return err
   658  					}
   659  					var count uint64
   660  					for _, pinfo := range taskPositions {
   661  						count += pinfo.Count
   662  					}
   663  					ts, _, err := pdCli.GetTS(ctx)
   664  					if err != nil {
   665  						return err
   666  					}
   667  					sinkGap := oracle.ExtractPhysical(status.ResolvedTs) - oracle.ExtractPhysical(status.CheckpointTs)
   668  					replicationGap := ts - oracle.ExtractPhysical(status.CheckpointTs)
   669  					statistics := profileStatus{
   670  						OPS:            (count - lastCount) / uint64(now.Unix()-lastTime.Unix()),
   671  						SinkGap:        fmt.Sprintf("%dms", sinkGap),
   672  						ReplicationGap: fmt.Sprintf("%dms", replicationGap),
   673  						Count:          count,
   674  					}
   675  					_ = jsonPrint(cmd, &statistics)
   676  					lastCount = count
   677  					lastTime = now
   678  				}
   679  			}
   680  		},
   681  	}
   682  	command.PersistentFlags().StringVarP(&changefeedID, "changefeed-id", "c", "", "Replication task (changefeed) ID")
   683  	command.PersistentFlags().UintVarP(&interval, "interval", "I", 10, "Interval for outputing the latest statistics")
   684  	_ = command.MarkPersistentFlagRequired("changefeed-id")
   685  	return command
   686  }
   687  
   688  func newCreateChangefeedCyclicCommand() *cobra.Command {
   689  	command := &cobra.Command{
   690  		Use:   "cyclic",
   691  		Short: "(Experimental) Utility about cyclic replication",
   692  	}
   693  	command.AddCommand(
   694  		&cobra.Command{
   695  			Use:   "create-marktables",
   696  			Short: "Create cyclic replication mark tables",
   697  			Long:  ``,
   698  			RunE: func(cmd *cobra.Command, args []string) error {
   699  				ctx := defaultContext
   700  
   701  				cfg := config.GetDefaultReplicaConfig()
   702  				if len(configFile) > 0 {
   703  					if err := verifyReplicaConfig(configFile, "TiCDC changefeed", cfg); err != nil {
   704  						return err
   705  					}
   706  				}
   707  				ts, logical, err := pdCli.GetTS(ctx)
   708  				if err != nil {
   709  					return err
   710  				}
   711  				startTs = oracle.ComposeTS(ts, logical)
   712  
   713  				_, eligibleTables, err := verifyTables(getCredential(), cfg, startTs)
   714  				if err != nil {
   715  					return err
   716  				}
   717  				tables := make([]mark.TableName, len(eligibleTables))
   718  				for i := range eligibleTables {
   719  					tables[i] = &eligibleTables[i]
   720  				}
   721  				err = mark.CreateMarkTables(ctx, cyclicUpstreamDSN, getUpstreamCredential(), tables...)
   722  				if err != nil {
   723  					return err
   724  				}
   725  				cmd.Printf("Create cyclic replication mark tables successfully! Total tables: %d\n", len(eligibleTables))
   726  				return nil
   727  			},
   728  		})
   729  	command.PersistentFlags().StringVar(&cyclicUpstreamDSN, "cyclic-upstream-dsn", "", "(Expremental) Upsteam TiDB DSN in the form of [user[:password]@][net[(addr)]]/")
   730  	command.PersistentFlags().StringVar(&upstreamSslCaPath, "cyclic-upstream-ssl-ca", "", "CA certificate path for TLS connection")
   731  	command.PersistentFlags().StringVar(&upstreamSslCertPath, "cyclic-upstream-ssl-cert", "", "Certificate path for TLS connection")
   732  	command.PersistentFlags().StringVar(&upstreamSslKeyPath, "cyclic-upstream-ssl-key", "", "Private key path for TLS connection")
   733  
   734  	return command
   735  }
   736  
   737  var (
   738  	upstreamSslCaPath   string
   739  	upstreamSslCertPath string
   740  	upstreamSslKeyPath  string
   741  )
   742  
   743  func getUpstreamCredential() *security.Credential {
   744  	return &security.Credential{
   745  		CAPath:   upstreamSslCaPath,
   746  		CertPath: upstreamSslCertPath,
   747  		KeyPath:  upstreamSslKeyPath,
   748  	}
   749  }