github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/model/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 model
    15  
    16  import (
    17  	"encoding/json"
    18  	"math"
    19  	"net/url"
    20  	"regexp"
    21  	"time"
    22  
    23  	"github.com/pingcap/errors"
    24  	"github.com/pingcap/log"
    25  	"github.com/pingcap/tiflow/pkg/config"
    26  	cerror "github.com/pingcap/tiflow/pkg/errors"
    27  	"github.com/pingcap/tiflow/pkg/sink"
    28  	"github.com/pingcap/tiflow/pkg/util"
    29  	"github.com/pingcap/tiflow/pkg/version"
    30  	"github.com/tikv/client-go/v2/oracle"
    31  	"go.uber.org/zap"
    32  )
    33  
    34  const (
    35  	// DefaultNamespace is the default namespace value,
    36  	// all the old changefeed will be put into default namespace
    37  	DefaultNamespace = "default"
    38  )
    39  
    40  // ChangeFeedID is the type for change feed ID
    41  type ChangeFeedID struct {
    42  	// Namespace and ID pair is unique in one ticdc cluster
    43  	// the default value of Namespace is "default"
    44  	Namespace string
    45  	ID        string
    46  }
    47  
    48  // String implements fmt.Stringer interface
    49  func (c ChangeFeedID) String() string {
    50  	return c.Namespace + "/" + c.ID
    51  }
    52  
    53  // DefaultChangeFeedID returns `ChangeFeedID` with default namespace
    54  func DefaultChangeFeedID(id string) ChangeFeedID {
    55  	return ChangeFeedID{
    56  		Namespace: DefaultNamespace,
    57  		ID:        id,
    58  	}
    59  }
    60  
    61  // ChangeFeedID4Test returns `ChangefeedID` with given namespace and id
    62  func ChangeFeedID4Test(namespace, id string) ChangeFeedID {
    63  	return ChangeFeedID{
    64  		Namespace: namespace,
    65  		ID:        id,
    66  	}
    67  }
    68  
    69  // SortEngine is the sorter engine
    70  type SortEngine = string
    71  
    72  // sort engines
    73  const (
    74  	SortInMemory SortEngine = "memory"
    75  	SortInFile   SortEngine = "file"
    76  	SortUnified  SortEngine = "unified"
    77  )
    78  
    79  // FeedState represents the running state of a changefeed
    80  type FeedState string
    81  
    82  // All FeedStates
    83  // Only `StateNormal` and `StatePending` changefeed is running,
    84  // others are stopped.
    85  const (
    86  	StateNormal   FeedState = "normal"
    87  	StatePending  FeedState = "pending"
    88  	StateFailed   FeedState = "failed"
    89  	StateStopped  FeedState = "stopped"
    90  	StateRemoved  FeedState = "removed"
    91  	StateFinished FeedState = "finished"
    92  	StateWarning  FeedState = "warning"
    93  	// StateUnInitialized is used for the changefeed that has not been initialized
    94  	// it only exists in memory for a short time and will not be persisted to storage
    95  	StateUnInitialized FeedState = ""
    96  )
    97  
    98  // ToInt return an int for each `FeedState`, only use this for metrics.
    99  func (s FeedState) ToInt() int {
   100  	switch s {
   101  	case StateNormal:
   102  		return 0
   103  	case StatePending:
   104  		return 1
   105  	case StateFailed:
   106  		return 2
   107  	case StateStopped:
   108  		return 3
   109  	case StateFinished:
   110  		return 4
   111  	case StateRemoved:
   112  		return 5
   113  	case StateWarning:
   114  		return 6
   115  	case StateUnInitialized:
   116  		return 7
   117  	}
   118  	// -1 for unknown feed state
   119  	return -1
   120  }
   121  
   122  // IsNeeded return true if the given feedState matches the listState.
   123  func (s FeedState) IsNeeded(need string) bool {
   124  	if need == "all" {
   125  		return true
   126  	}
   127  	if need == "" {
   128  		switch s {
   129  		case StateNormal:
   130  			return true
   131  		case StateStopped:
   132  			return true
   133  		case StateFailed:
   134  			return true
   135  		case StateWarning:
   136  			return true
   137  		case StatePending:
   138  			return true
   139  		}
   140  	}
   141  	return need == string(s)
   142  }
   143  
   144  // IsRunning return true if the feedState represents a running state.
   145  func (s FeedState) IsRunning() bool {
   146  	return s == StateNormal || s == StateWarning
   147  }
   148  
   149  // ChangeFeedInfo describes the detail of a ChangeFeed
   150  type ChangeFeedInfo struct {
   151  	UpstreamID uint64    `json:"upstream-id"`
   152  	Namespace  string    `json:"namespace"`
   153  	ID         string    `json:"changefeed-id"`
   154  	SinkURI    string    `json:"sink-uri"`
   155  	CreateTime time.Time `json:"create-time"`
   156  	// Start sync at this commit ts if `StartTs` is specify or using the CreateTime of changefeed.
   157  	StartTs uint64 `json:"start-ts"`
   158  	// The ChangeFeed will exits until sync to timestamp TargetTs
   159  	TargetTs uint64 `json:"target-ts"`
   160  	// used for admin job notification, trigger watch event in capture
   161  	AdminJobType AdminJobType `json:"admin-job-type"`
   162  	Engine       SortEngine   `json:"sort-engine"`
   163  	// SortDir is deprecated
   164  	// it cannot be set by user in changefeed level, any assignment to it should be ignored.
   165  	// but can be fetched for backward compatibility
   166  	SortDir string `json:"sort-dir"`
   167  
   168  	Config  *config.ReplicaConfig `json:"config"`
   169  	State   FeedState             `json:"state"`
   170  	Error   *RunningError         `json:"error"`
   171  	Warning *RunningError         `json:"warning"`
   172  
   173  	CreatorVersion string `json:"creator-version"`
   174  	// Epoch is the epoch of a changefeed, changes on every restart.
   175  	Epoch uint64 `json:"epoch"`
   176  }
   177  
   178  const changeFeedIDMaxLen = 128
   179  
   180  var changeFeedIDRe = regexp.MustCompile(`^[a-zA-Z0-9]+(-[a-zA-Z0-9]+)*$`)
   181  
   182  // ValidateChangefeedID returns true if the changefeed ID matches
   183  // the pattern "^[a-zA-Z0-9]+(\-[a-zA-Z0-9]+)*$", length no more than "changeFeedIDMaxLen", eg, "simple-changefeed-task".
   184  func ValidateChangefeedID(changefeedID string) error {
   185  	if !changeFeedIDRe.MatchString(changefeedID) || len(changefeedID) > changeFeedIDMaxLen {
   186  		return cerror.ErrInvalidChangefeedID.GenWithStackByArgs(changeFeedIDMaxLen)
   187  	}
   188  	return nil
   189  }
   190  
   191  const namespaceMaxLen = 128
   192  
   193  var namespaceRe = regexp.MustCompile(`^[a-zA-Z0-9]+(-[a-zA-Z0-9]+)*$`)
   194  
   195  // ValidateNamespace returns true if the namespace matches
   196  // the pattern "^[a-zA-Z0-9]+(\-[a-zA-Z0-9]+)*$",
   197  // length no more than "changeFeedIDMaxLen", eg, "simple-changefeed-task".
   198  func ValidateNamespace(namespace string) error {
   199  	if !namespaceRe.MatchString(namespace) || len(namespace) > namespaceMaxLen {
   200  		return cerror.ErrInvalidNamespace.GenWithStackByArgs(namespaceRe)
   201  	}
   202  	return nil
   203  }
   204  
   205  // NeedBlockGC returns true if the changefeed need to block the GC safepoint.
   206  // Note: if the changefeed is failed by GC, it should not block the GC safepoint.
   207  func (info *ChangeFeedInfo) NeedBlockGC() bool {
   208  	switch info.State {
   209  	case StateNormal, StateStopped, StatePending, StateWarning:
   210  		return true
   211  	case StateFailed:
   212  		return !info.isFailedByGC()
   213  	case StateFinished, StateRemoved:
   214  	default:
   215  	}
   216  	return false
   217  }
   218  
   219  func (info *ChangeFeedInfo) isFailedByGC() bool {
   220  	if info.Error == nil {
   221  		log.Panic("changefeed info is not consistent",
   222  			zap.Any("state", info.State), zap.Any("error", info.Error))
   223  	}
   224  	return cerror.IsChangefeedGCFastFailErrorCode(errors.RFCErrorCode(info.Error.Code))
   225  }
   226  
   227  // String implements fmt.Stringer interface, but hide some sensitive information
   228  func (info *ChangeFeedInfo) String() (str string) {
   229  	var err error
   230  	str, err = info.Marshal()
   231  	if err != nil {
   232  		log.Error("failed to marshal changefeed info", zap.Error(err))
   233  		return
   234  	}
   235  	clone := new(ChangeFeedInfo)
   236  	err = clone.Unmarshal([]byte(str))
   237  	if err != nil {
   238  		log.Error("failed to unmarshal changefeed info", zap.Error(err))
   239  		return
   240  	}
   241  
   242  	clone.SinkURI = util.MaskSensitiveDataInURI(clone.SinkURI)
   243  	if clone.Config != nil {
   244  		clone.Config.MaskSensitiveData()
   245  	}
   246  
   247  	str, err = clone.Marshal()
   248  	if err != nil {
   249  		log.Error("failed to marshal changefeed info", zap.Error(err))
   250  	}
   251  	return
   252  }
   253  
   254  // GetStartTs returns StartTs if it's specified or using the
   255  // CreateTime of changefeed.
   256  func (info *ChangeFeedInfo) GetStartTs() uint64 {
   257  	if info.StartTs > 0 {
   258  		return info.StartTs
   259  	}
   260  
   261  	return oracle.GoTimeToTS(info.CreateTime)
   262  }
   263  
   264  // GetCheckpointTs returns CheckpointTs if it's specified in ChangeFeedStatus, otherwise StartTs is returned.
   265  func (info *ChangeFeedInfo) GetCheckpointTs(status *ChangeFeedStatus) uint64 {
   266  	if status != nil {
   267  		return status.CheckpointTs
   268  	}
   269  	return info.GetStartTs()
   270  }
   271  
   272  // GetTargetTs returns TargetTs if it's specified, otherwise MaxUint64 is returned.
   273  func (info *ChangeFeedInfo) GetTargetTs() uint64 {
   274  	if info.TargetTs > 0 {
   275  		return info.TargetTs
   276  	}
   277  	return uint64(math.MaxUint64)
   278  }
   279  
   280  // Marshal returns the json marshal format of a ChangeFeedInfo
   281  func (info *ChangeFeedInfo) Marshal() (string, error) {
   282  	data, err := json.Marshal(info)
   283  	return string(data), cerror.WrapError(cerror.ErrMarshalFailed, err)
   284  }
   285  
   286  // Unmarshal unmarshals into *ChangeFeedInfo from json marshal byte slice
   287  func (info *ChangeFeedInfo) Unmarshal(data []byte) error {
   288  	err := json.Unmarshal(data, &info)
   289  	if err != nil {
   290  		return errors.Annotatef(
   291  			cerror.WrapError(cerror.ErrUnmarshalFailed, err), "Unmarshal data: %v", data)
   292  	}
   293  	return nil
   294  }
   295  
   296  // Clone returns a cloned ChangeFeedInfo
   297  func (info *ChangeFeedInfo) Clone() (*ChangeFeedInfo, error) {
   298  	s, err := info.Marshal()
   299  	if err != nil {
   300  		return nil, err
   301  	}
   302  	cloned := new(ChangeFeedInfo)
   303  	err = cloned.Unmarshal([]byte(s))
   304  	return cloned, err
   305  }
   306  
   307  // VerifyAndComplete verifies changefeed info and may fill in some fields.
   308  // If a required field is not provided, return an error.
   309  // If some necessary filed is missing but can use a default value, fill in it.
   310  func (info *ChangeFeedInfo) VerifyAndComplete() {
   311  	defaultConfig := config.GetDefaultReplicaConfig()
   312  	if info.Engine == "" {
   313  		info.Engine = SortUnified
   314  	}
   315  	if info.Config.Filter == nil {
   316  		info.Config.Filter = defaultConfig.Filter
   317  	}
   318  	if info.Config.Mounter == nil {
   319  		info.Config.Mounter = defaultConfig.Mounter
   320  	}
   321  	if info.Config.Sink == nil {
   322  		info.Config.Sink = defaultConfig.Sink
   323  	}
   324  	if info.Config.Consistent == nil {
   325  		info.Config.Consistent = defaultConfig.Consistent
   326  	}
   327  	if info.Config.Scheduler == nil {
   328  		info.Config.Scheduler = defaultConfig.Scheduler
   329  	}
   330  
   331  	if info.Config.Integrity == nil {
   332  		info.Config.Integrity = defaultConfig.Integrity
   333  	}
   334  	if info.Config.ChangefeedErrorStuckDuration == nil {
   335  		info.Config.ChangefeedErrorStuckDuration = defaultConfig.ChangefeedErrorStuckDuration
   336  	}
   337  	if info.Config.SyncedStatus == nil {
   338  		info.Config.SyncedStatus = defaultConfig.SyncedStatus
   339  	}
   340  	info.RmUnusedFields()
   341  }
   342  
   343  // RmUnusedFields removes unnecessary fields based on the downstream type and
   344  // the protocol. Since we utilize a common changefeed configuration template,
   345  // certain fields may not be utilized for certain protocols.
   346  func (info *ChangeFeedInfo) RmUnusedFields() {
   347  	uri, err := url.Parse(info.SinkURI)
   348  	if err != nil {
   349  		log.Warn(
   350  			"failed to parse the sink uri",
   351  			zap.Error(err),
   352  			zap.Any("sinkUri", info.SinkURI),
   353  		)
   354  		return
   355  	}
   356  	// blackhole is for testing purpose, no need to remove fields
   357  	if sink.IsBlackHoleScheme(uri.Scheme) {
   358  		return
   359  	}
   360  	if !sink.IsMQScheme(uri.Scheme) {
   361  		info.rmMQOnlyFields()
   362  	} else {
   363  		// remove schema registry for MQ downstream with
   364  		// protocol other than avro
   365  		if util.GetOrZero(info.Config.Sink.Protocol) != config.ProtocolAvro.String() {
   366  			info.Config.Sink.SchemaRegistry = nil
   367  		}
   368  	}
   369  
   370  	if !sink.IsStorageScheme(uri.Scheme) {
   371  		info.rmStorageOnlyFields()
   372  	}
   373  
   374  	if !sink.IsMySQLCompatibleScheme(uri.Scheme) {
   375  		info.rmDBOnlyFields()
   376  	} else {
   377  		// remove fields only being used by MQ and Storage downstream
   378  		info.Config.Sink.Protocol = nil
   379  		info.Config.Sink.Terminator = nil
   380  	}
   381  }
   382  
   383  func (info *ChangeFeedInfo) rmMQOnlyFields() {
   384  	log.Info("since the downstream is not a MQ, remove MQ only fields",
   385  		zap.String("namespace", info.Namespace),
   386  		zap.String("changefeed", info.ID))
   387  	info.Config.Sink.DispatchRules = nil
   388  	info.Config.Sink.SchemaRegistry = nil
   389  	info.Config.Sink.EncoderConcurrency = nil
   390  	info.Config.Sink.EnableKafkaSinkV2 = nil
   391  	info.Config.Sink.OnlyOutputUpdatedColumns = nil
   392  	info.Config.Sink.DeleteOnlyOutputHandleKeyColumns = nil
   393  	info.Config.Sink.ContentCompatible = nil
   394  	info.Config.Sink.KafkaConfig = nil
   395  }
   396  
   397  func (info *ChangeFeedInfo) rmStorageOnlyFields() {
   398  	info.Config.Sink.CSVConfig = nil
   399  	info.Config.Sink.DateSeparator = nil
   400  	info.Config.Sink.EnablePartitionSeparator = nil
   401  	info.Config.Sink.FileIndexWidth = nil
   402  	info.Config.Sink.CloudStorageConfig = nil
   403  }
   404  
   405  func (info *ChangeFeedInfo) rmDBOnlyFields() {
   406  	info.Config.EnableSyncPoint = nil
   407  	info.Config.BDRMode = nil
   408  	info.Config.SyncPointInterval = nil
   409  	info.Config.SyncPointRetention = nil
   410  	info.Config.Consistent = nil
   411  	info.Config.Sink.SafeMode = nil
   412  	info.Config.Sink.MySQLConfig = nil
   413  }
   414  
   415  // FixIncompatible fixes incompatible changefeed meta info.
   416  func (info *ChangeFeedInfo) FixIncompatible() {
   417  	creatorVersionGate := version.NewCreatorVersionGate(info.CreatorVersion)
   418  	if creatorVersionGate.ChangefeedStateFromAdminJob() {
   419  		log.Info("Start fixing incompatible changefeed state", zap.String("changefeed", info.String()))
   420  		info.fixState()
   421  		log.Info("Fix incompatibility changefeed state completed", zap.String("changefeed", info.String()))
   422  	}
   423  
   424  	if creatorVersionGate.ChangefeedAcceptUnknownProtocols() {
   425  		log.Info("Start fixing incompatible changefeed MQ sink protocol", zap.String("changefeed", info.String()))
   426  		info.fixMQSinkProtocol()
   427  		log.Info("Fix incompatibility changefeed MQ sink protocol completed", zap.String("changefeed", info.String()))
   428  	}
   429  
   430  	if creatorVersionGate.ChangefeedAcceptProtocolInMysqlSinURI() {
   431  		log.Info("Start fixing incompatible changefeed sink uri", zap.String("changefeed", info.String()))
   432  		info.fixMySQLSinkProtocol()
   433  		log.Info("Fix incompatibility changefeed sink uri completed", zap.String("changefeed", info.String()))
   434  	}
   435  
   436  	if info.Config.MemoryQuota == uint64(0) {
   437  		log.Info("Start fixing incompatible memory quota", zap.String("changefeed", info.String()))
   438  		info.fixMemoryQuota()
   439  		log.Info("Fix incompatible memory quota completed", zap.String("changefeed", info.String()))
   440  	}
   441  
   442  	if info.Config.ChangefeedErrorStuckDuration == nil {
   443  		log.Info("Start fixing incompatible error stuck duration", zap.String("changefeed", info.String()))
   444  		info.Config.ChangefeedErrorStuckDuration = config.GetDefaultReplicaConfig().ChangefeedErrorStuckDuration
   445  		log.Info("Fix incompatible error stuck duration completed", zap.String("changefeed", info.String()))
   446  	}
   447  
   448  	log.Info("Start fixing incompatible scheduler", zap.String("changefeed", info.String()))
   449  	inheritV66 := creatorVersionGate.ChangefeedInheritSchedulerConfigFromV66()
   450  	info.fixScheduler(inheritV66)
   451  	log.Info("Fix incompatible scheduler completed", zap.String("changefeed", info.String()))
   452  }
   453  
   454  // fixState attempts to fix state loss from upgrading the old owner to the new owner.
   455  func (info *ChangeFeedInfo) fixState() {
   456  	// Notice: In the old owner we used AdminJobType field to determine if the task was paused or not,
   457  	// we need to handle this field in the new owner.
   458  	// Otherwise, we will see that the old version of the task is paused and then upgraded,
   459  	// and the task is automatically resumed after the upgrade.
   460  	state := info.State
   461  	// Upgrading from an old owner, we need to deal with cases where the state is normal,
   462  	// but actually contains errors and does not match the admin job type.
   463  	if state == StateNormal {
   464  		switch info.AdminJobType {
   465  		// This corresponds to the case of failure or error.
   466  		case AdminNone, AdminResume:
   467  			if info.Error != nil {
   468  				if cerror.IsChangefeedGCFastFailErrorCode(errors.RFCErrorCode(info.Error.Code)) {
   469  					state = StateFailed
   470  				} else {
   471  					state = StateWarning
   472  				}
   473  			}
   474  		case AdminStop:
   475  			state = StateStopped
   476  		case AdminFinish:
   477  			state = StateFinished
   478  		case AdminRemove:
   479  			state = StateRemoved
   480  		}
   481  	}
   482  
   483  	if state != info.State {
   484  		log.Info("handle old owner inconsistent state",
   485  			zap.String("oldState", string(info.State)),
   486  			zap.String("adminJob", info.AdminJobType.String()),
   487  			zap.String("newState", string(state)))
   488  		info.State = state
   489  	}
   490  }
   491  
   492  func (info *ChangeFeedInfo) fixMySQLSinkProtocol() {
   493  	uri, err := url.Parse(info.SinkURI)
   494  	if err != nil {
   495  		log.Warn("parse sink URI failed", zap.Error(err))
   496  		// SAFETY: It is safe to ignore this unresolvable sink URI here,
   497  		// as it is almost impossible for this to happen.
   498  		// If we ignore it when fixing it after it happens,
   499  		// it will expose the problem when starting the changefeed,
   500  		// which is easier to troubleshoot than reporting the error directly in the bootstrap process.
   501  		return
   502  	}
   503  
   504  	if sink.IsMQScheme(uri.Scheme) {
   505  		return
   506  	}
   507  
   508  	query := uri.Query()
   509  	protocolStr := query.Get(config.ProtocolKey)
   510  	if protocolStr != "" || info.Config.Sink.Protocol != nil {
   511  		maskedSinkURI, _ := util.MaskSinkURI(info.SinkURI)
   512  		log.Warn("sink URI or sink config contains protocol, but scheme is not mq",
   513  			zap.String("sinkURI", maskedSinkURI),
   514  			zap.String("protocol", protocolStr),
   515  			zap.Any("sinkConfig", info.Config.Sink))
   516  		// always set protocol of mysql sink to ""
   517  		query.Del(config.ProtocolKey)
   518  		info.updateSinkURIAndConfigProtocol(uri, "", query)
   519  	}
   520  }
   521  
   522  func (info *ChangeFeedInfo) fixMQSinkProtocol() {
   523  	uri, err := url.Parse(info.SinkURI)
   524  	if err != nil {
   525  		log.Warn("parse sink URI failed", zap.Error(err))
   526  		return
   527  	}
   528  
   529  	if !sink.IsMQScheme(uri.Scheme) {
   530  		return
   531  	}
   532  
   533  	needsFix := func(protocolStr string) bool {
   534  		_, err := config.ParseSinkProtocolFromString(protocolStr)
   535  		// There are two cases:
   536  		// 1. there is an error indicating that the old ticdc accepts
   537  		//    a protocol that is not known. It needs to be fixed as open protocol.
   538  		// 2. If it is default, then it needs to be fixed as open protocol.
   539  		return err != nil || protocolStr == config.ProtocolDefault.String()
   540  	}
   541  
   542  	query := uri.Query()
   543  	protocol := query.Get(config.ProtocolKey)
   544  	openProtocol := config.ProtocolOpen.String()
   545  
   546  	// The sinkURI always has a higher priority.
   547  	if protocol != "" && needsFix(protocol) {
   548  		query.Set(config.ProtocolKey, openProtocol)
   549  		info.updateSinkURIAndConfigProtocol(uri, openProtocol, query)
   550  		return
   551  	}
   552  
   553  	if needsFix(util.GetOrZero(info.Config.Sink.Protocol)) {
   554  		log.Info("handle incompatible protocol from sink config",
   555  			zap.String("oldProtocol", util.GetOrZero(info.Config.Sink.Protocol)),
   556  			zap.String("fixedProtocol", openProtocol))
   557  		info.Config.Sink.Protocol = util.AddressOf(openProtocol)
   558  	}
   559  }
   560  
   561  func (info *ChangeFeedInfo) updateSinkURIAndConfigProtocol(uri *url.URL, newProtocol string, newQuery url.Values) {
   562  	newRawQuery := newQuery.Encode()
   563  	maskedURI, _ := util.MaskSinkURI(uri.String())
   564  	log.Info("handle incompatible protocol from sink URI",
   565  		zap.String("oldURI", maskedURI),
   566  		zap.String("newProtocol", newProtocol))
   567  
   568  	uri.RawQuery = newRawQuery
   569  	fixedSinkURI := uri.String()
   570  	info.SinkURI = fixedSinkURI
   571  	info.Config.Sink.Protocol = util.AddressOf(newProtocol)
   572  }
   573  
   574  func (info *ChangeFeedInfo) fixMemoryQuota() {
   575  	info.Config.FixMemoryQuota()
   576  }
   577  
   578  func (info *ChangeFeedInfo) fixScheduler(inheritV66 bool) {
   579  	info.Config.FixScheduler(inheritV66)
   580  }
   581  
   582  // ChangeFeedStatusForAPI uses to transfer the status of changefeed for API.
   583  type ChangeFeedStatusForAPI struct {
   584  	ResolvedTs   uint64 `json:"resolved-ts"`
   585  	CheckpointTs uint64 `json:"checkpoint-ts"`
   586  }
   587  
   588  // ChangeFeedSyncedStatusForAPI uses to transfer the synced status of changefeed for API.
   589  type ChangeFeedSyncedStatusForAPI struct {
   590  	CheckpointTs        uint64 `json:"checkpoint-ts"`
   591  	LastSyncedTs        uint64 `json:"last-sync-time"`
   592  	PullerResolvedTs    uint64 `json:"puller-resolved-ts"`
   593  	SyncedCheckInterval int64  `json:"synced-check-interval"`
   594  	CheckpointInterval  int64  `json:"checkpoint-interval"`
   595  }