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

     1  // Copyright 2021 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 v1
    15  
    16  import (
    17  	"context"
    18  	"net/url"
    19  	"time"
    20  
    21  	"github.com/pingcap/errors"
    22  	"github.com/pingcap/log"
    23  	"github.com/pingcap/tiflow/cdc/controller"
    24  	"github.com/pingcap/tiflow/cdc/entry"
    25  	"github.com/pingcap/tiflow/cdc/model"
    26  	"github.com/pingcap/tiflow/cdc/owner"
    27  	"github.com/pingcap/tiflow/cdc/sink/validator"
    28  	"github.com/pingcap/tiflow/pkg/config"
    29  	cerror "github.com/pingcap/tiflow/pkg/errors"
    30  	"github.com/pingcap/tiflow/pkg/etcd"
    31  	"github.com/pingcap/tiflow/pkg/filter"
    32  	"github.com/pingcap/tiflow/pkg/txnutil/gc"
    33  	"github.com/pingcap/tiflow/pkg/upstream"
    34  	"github.com/pingcap/tiflow/pkg/util"
    35  	"github.com/pingcap/tiflow/pkg/version"
    36  	"github.com/r3labs/diff"
    37  	"github.com/tikv/client-go/v2/oracle"
    38  )
    39  
    40  // verifyCreateChangefeedConfig verifies ChangefeedConfig for create a changefeed
    41  func verifyCreateChangefeedConfig(
    42  	ctx context.Context,
    43  	changefeedConfig model.ChangefeedConfig,
    44  	up *upstream.Upstream,
    45  	ctrl controller.Controller,
    46  	ectdClient etcd.CDCEtcdClient,
    47  ) (*model.ChangeFeedInfo, error) {
    48  	// verify sinkURI
    49  	if changefeedConfig.SinkURI == "" {
    50  		return nil, cerror.ErrSinkURIInvalid.GenWithStackByArgs("sink-uri is empty, can't not create a changefeed without sink-uri")
    51  	}
    52  
    53  	// verify changefeedID
    54  	if err := model.ValidateChangefeedID(changefeedConfig.ID); err != nil {
    55  		return nil, cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedConfig.ID)
    56  	}
    57  	// check if the changefeed exists
    58  	ok, err := ctrl.IsChangefeedExists(ctx, model.DefaultChangeFeedID(changefeedConfig.ID))
    59  	if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) {
    60  		return nil, err
    61  	}
    62  	if ok {
    63  		return nil, cerror.ErrChangeFeedAlreadyExists.GenWithStackByArgs(changefeedConfig.ID)
    64  	}
    65  
    66  	ts, logical, err := up.PDClient.GetTS(ctx)
    67  	if err != nil {
    68  		return nil, cerror.ErrPDEtcdAPIError.GenWithStackByArgs("fail to get ts from pd client")
    69  	}
    70  	currentTSO := oracle.ComposeTS(ts, logical)
    71  	// verify start-ts
    72  	if changefeedConfig.StartTS == 0 {
    73  		changefeedConfig.StartTS = currentTSO
    74  	} else if changefeedConfig.StartTS > currentTSO {
    75  		return nil, cerror.ErrAPIInvalidParam.GenWithStack(
    76  			"invalid start-ts %v, larger than current tso %v", changefeedConfig.StartTS, currentTSO)
    77  	}
    78  
    79  	// Ensure the start ts is valid in the next 1 hour.
    80  	const ensureTTL = 60 * 60
    81  	if err := gc.EnsureChangefeedStartTsSafety(
    82  		ctx,
    83  		up.PDClient,
    84  		ectdClient.GetEnsureGCServiceID(gc.EnsureGCServiceCreating),
    85  		model.DefaultChangeFeedID(changefeedConfig.ID),
    86  		ensureTTL, changefeedConfig.StartTS); err != nil {
    87  		if !cerror.ErrStartTsBeforeGC.Equal(err) {
    88  			return nil, cerror.ErrPDEtcdAPIError.Wrap(err)
    89  		}
    90  		return nil, err
    91  	}
    92  
    93  	// verify target-ts
    94  	if changefeedConfig.TargetTS > 0 && changefeedConfig.TargetTS <= changefeedConfig.StartTS {
    95  		return nil, cerror.ErrTargetTsBeforeStartTs.GenWithStackByArgs(changefeedConfig.TargetTS, changefeedConfig.StartTS)
    96  	}
    97  
    98  	// init replicaConfig
    99  	replicaConfig := config.GetDefaultReplicaConfig()
   100  	replicaConfig.ForceReplicate = changefeedConfig.ForceReplicate
   101  	if changefeedConfig.MounterWorkerNum != 0 {
   102  		replicaConfig.Mounter.WorkerNum = changefeedConfig.MounterWorkerNum
   103  	}
   104  	if changefeedConfig.SinkConfig != nil {
   105  		replicaConfig.Sink = changefeedConfig.SinkConfig
   106  	}
   107  	if len(changefeedConfig.IgnoreTxnStartTs) != 0 {
   108  		replicaConfig.Filter.IgnoreTxnStartTs = changefeedConfig.IgnoreTxnStartTs
   109  	}
   110  	if len(changefeedConfig.FilterRules) != 0 {
   111  		replicaConfig.Filter.Rules = changefeedConfig.FilterRules
   112  	}
   113  	// verify replicaConfig
   114  	sinkURIParsed, err := url.Parse(changefeedConfig.SinkURI)
   115  	if err != nil {
   116  		return nil, cerror.WrapError(cerror.ErrSinkURIInvalid, err)
   117  	}
   118  	err = replicaConfig.ValidateAndAdjust(sinkURIParsed)
   119  	if err != nil {
   120  		return nil, err
   121  	}
   122  
   123  	captureInfos, err := ctrl.GetCaptures(ctx)
   124  	if err != nil {
   125  		return nil, err
   126  	}
   127  	// set sortEngine
   128  	cdcClusterVer, err := version.GetTiCDCClusterVersion(model.ListVersionsFromCaptureInfos(captureInfos))
   129  	if err != nil {
   130  		return nil, err
   131  	}
   132  	sortEngine := model.SortUnified
   133  	if !cdcClusterVer.LessThan500RC() {
   134  		log.Warn("The TiCDC cluster is built from unknown branch or less than 5.0.0-rc, the old-value are disabled by default.")
   135  		if !cdcClusterVer.ShouldEnableUnifiedSorterByDefault() {
   136  			sortEngine = model.SortInMemory
   137  		}
   138  	}
   139  
   140  	// init ChangefeedInfo
   141  	info := &model.ChangeFeedInfo{
   142  		Namespace:      model.DefaultNamespace,
   143  		ID:             changefeedConfig.ID,
   144  		UpstreamID:     up.ID,
   145  		SinkURI:        changefeedConfig.SinkURI,
   146  		CreateTime:     time.Now(),
   147  		StartTs:        changefeedConfig.StartTS,
   148  		TargetTs:       changefeedConfig.TargetTS,
   149  		Config:         replicaConfig,
   150  		Engine:         sortEngine,
   151  		State:          model.StateNormal,
   152  		CreatorVersion: version.ReleaseVersion,
   153  		Epoch:          owner.GenerateChangefeedEpoch(ctx, up.PDClient),
   154  	}
   155  	f, err := filter.NewFilter(replicaConfig, "")
   156  	if err != nil {
   157  		return nil, err
   158  	}
   159  	tableInfos, ineligibleTables, _, err := entry.VerifyTables(f,
   160  		up.KVStorage, changefeedConfig.StartTS)
   161  	if err != nil {
   162  		return nil, err
   163  	}
   164  	err = f.Verify(tableInfos)
   165  	if err != nil {
   166  		return nil, err
   167  	}
   168  	if !replicaConfig.ForceReplicate && !changefeedConfig.IgnoreIneligibleTable {
   169  		if len(ineligibleTables) != 0 {
   170  			return nil, cerror.ErrTableIneligible.GenWithStackByArgs(ineligibleTables)
   171  		}
   172  	}
   173  
   174  	_, err = util.GetTimezone(changefeedConfig.TimeZone)
   175  	if err != nil {
   176  		return nil, cerror.ErrAPIInvalidParam.Wrap(errors.Annotatef(err, "invalid timezone:%s", changefeedConfig.TimeZone))
   177  	}
   178  	if err := validator.Validate(ctx,
   179  		model.ChangeFeedID{Namespace: changefeedConfig.Namespace, ID: changefeedConfig.ID},
   180  		info.SinkURI, info.Config, up.PDClock,
   181  	); err != nil {
   182  		return nil, err
   183  	}
   184  
   185  	return info, nil
   186  }
   187  
   188  // VerifyUpdateChangefeedConfig verify ChangefeedConfig for update a changefeed
   189  func VerifyUpdateChangefeedConfig(ctx context.Context,
   190  	changefeedConfig model.ChangefeedConfig,
   191  	oldInfo *model.ChangeFeedInfo,
   192  ) (*model.ChangeFeedInfo, error) {
   193  	newInfo, err := oldInfo.Clone()
   194  	if err != nil {
   195  		return nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByArgs(err.Error())
   196  	}
   197  	// verify target_ts
   198  	if changefeedConfig.TargetTS != 0 {
   199  		if changefeedConfig.TargetTS <= newInfo.StartTs {
   200  			return nil, cerror.ErrChangefeedUpdateRefused.GenWithStack("can not update target-ts:%d less than start-ts:%d", changefeedConfig.TargetTS, newInfo.StartTs)
   201  		}
   202  		newInfo.TargetTs = changefeedConfig.TargetTS
   203  	}
   204  
   205  	// verify rules
   206  	if len(changefeedConfig.FilterRules) != 0 {
   207  		newInfo.Config.Filter.Rules = changefeedConfig.FilterRules
   208  		_, err = filter.VerifyTableRules(newInfo.Config.Filter)
   209  		if err != nil {
   210  			return nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByArgs(err.Error())
   211  		}
   212  	}
   213  
   214  	var sinkConfigUpdated, sinkURIUpdated bool
   215  	if len(changefeedConfig.IgnoreTxnStartTs) != 0 {
   216  		newInfo.Config.Filter.IgnoreTxnStartTs = changefeedConfig.IgnoreTxnStartTs
   217  	}
   218  	if changefeedConfig.MounterWorkerNum != 0 {
   219  		newInfo.Config.Mounter.WorkerNum = changefeedConfig.MounterWorkerNum
   220  	}
   221  	if changefeedConfig.SinkConfig != nil {
   222  		sinkConfigUpdated = true
   223  		newInfo.Config.Sink = changefeedConfig.SinkConfig
   224  	}
   225  	if changefeedConfig.SinkURI != "" {
   226  		sinkURIUpdated = true
   227  		newInfo.SinkURI = changefeedConfig.SinkURI
   228  	}
   229  
   230  	if sinkConfigUpdated || sinkURIUpdated {
   231  		// check sink config is compatible with sinkURI
   232  		newCfg := newInfo.Config.Sink
   233  		oldCfg := oldInfo.Config.Sink
   234  		err := newCfg.CheckCompatibilityWithSinkURI(oldCfg, newInfo.SinkURI)
   235  		if err != nil {
   236  			return nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByCause(err)
   237  		}
   238  
   239  		if err := validator.Validate(ctx,
   240  			model.ChangeFeedID{Namespace: changefeedConfig.Namespace, ID: changefeedConfig.ID},
   241  			newInfo.SinkURI, newInfo.Config, nil); err != nil {
   242  			return nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByCause(err)
   243  		}
   244  	}
   245  
   246  	if !diff.Changed(oldInfo, newInfo) {
   247  		return nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByArgs("changefeed config is the same with the old one, do nothing")
   248  	}
   249  
   250  	return newInfo, nil
   251  }