github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/api/v2/api_helpers.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  	"crypto/tls"
    19  	"net/url"
    20  	"strings"
    21  	"time"
    22  
    23  	"github.com/google/uuid"
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/log"
    26  	tidbkv "github.com/pingcap/tidb/pkg/kv"
    27  	"github.com/pingcap/tiflow/cdc/controller"
    28  	"github.com/pingcap/tiflow/cdc/entry"
    29  	"github.com/pingcap/tiflow/cdc/kv"
    30  	"github.com/pingcap/tiflow/cdc/model"
    31  	"github.com/pingcap/tiflow/cdc/owner"
    32  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher"
    33  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/transformer/columnselector"
    34  	"github.com/pingcap/tiflow/cdc/sink/validator"
    35  	"github.com/pingcap/tiflow/pkg/config"
    36  	cerror "github.com/pingcap/tiflow/pkg/errors"
    37  	"github.com/pingcap/tiflow/pkg/filter"
    38  	"github.com/pingcap/tiflow/pkg/security"
    39  	"github.com/pingcap/tiflow/pkg/sink"
    40  	"github.com/pingcap/tiflow/pkg/txnutil/gc"
    41  	"github.com/pingcap/tiflow/pkg/version"
    42  	"github.com/r3labs/diff"
    43  	"github.com/tikv/client-go/v2/oracle"
    44  	pd "github.com/tikv/pd/client"
    45  	"go.etcd.io/etcd/client/pkg/v3/logutil"
    46  	clientv3 "go.etcd.io/etcd/client/v3"
    47  	"go.uber.org/zap"
    48  	"go.uber.org/zap/zapcore"
    49  	"google.golang.org/grpc"
    50  	"google.golang.org/grpc/backoff"
    51  )
    52  
    53  // RegisterImportTaskPrefix denotes the key prefix associated with the entries
    54  // containning import/restore information in the embedded Etcd of the
    55  // upstream PD.
    56  const RegisterImportTaskPrefix = "/tidb/brie/import"
    57  
    58  // APIV2Helpers is a collections of helper functions of OpenAPIV2.
    59  // Defining it as an interface to make APIs more testable.
    60  type APIV2Helpers interface {
    61  	// verifyCreateChangefeedConfig verifies the changefeedConfig,
    62  	// and yield a valid changefeedInfo or error
    63  	verifyCreateChangefeedConfig(
    64  		ctx context.Context,
    65  		cfg *ChangefeedConfig,
    66  		pdClient pd.Client,
    67  		ctrl controller.Controller,
    68  		ensureGCServiceID string,
    69  		kvStorage tidbkv.Storage,
    70  	) (*model.ChangeFeedInfo, error)
    71  
    72  	// verifyUpdateChangefeedConfig verifies the changefeed update config,
    73  	// and returns a pair of valid changefeedInfo & upstreamInfo
    74  	verifyUpdateChangefeedConfig(
    75  		ctx context.Context,
    76  		cfg *ChangefeedConfig,
    77  		oldInfo *model.ChangeFeedInfo,
    78  		oldUpInfo *model.UpstreamInfo,
    79  		kvStorage tidbkv.Storage,
    80  		checkpointTs uint64,
    81  	) (*model.ChangeFeedInfo, *model.UpstreamInfo, error)
    82  
    83  	// verifyUpstream verifies the upstreamConfig
    84  	verifyUpstream(
    85  		ctx context.Context,
    86  		changefeedConfig *ChangefeedConfig,
    87  		cfInfo *model.ChangeFeedInfo,
    88  	) error
    89  
    90  	verifyResumeChangefeedConfig(
    91  		ctx context.Context,
    92  		pdClient pd.Client,
    93  		gcServiceID string,
    94  		changefeedID model.ChangeFeedID,
    95  		overrideCheckpointTs uint64,
    96  	) error
    97  
    98  	// getPDClient returns a PDClient given the PD cluster addresses and a credential
    99  	getPDClient(
   100  		ctx context.Context,
   101  		pdAddrs []string,
   102  		credential *security.Credential,
   103  	) (pd.Client, error)
   104  
   105  	// getEtcdClient returns an Etcd client given the PD endpoints and
   106  	// tls config
   107  	getEtcdClient(
   108  		pdAddrs []string,
   109  		tlsConfig *tls.Config,
   110  	) (*clientv3.Client, error)
   111  
   112  	// getKVCreateTiStore wraps kv.createTiStore method to increase testability
   113  	createTiStore(
   114  		pdAddrs []string,
   115  		credential *security.Credential,
   116  	) (tidbkv.Storage, error)
   117  
   118  	// getVerifiedTables wraps entry.VerifyTables to increase testability
   119  	getVerifiedTables(replicaConfig *config.ReplicaConfig,
   120  		storage tidbkv.Storage, startTs uint64,
   121  		scheme string, topic string, protocol config.Protocol,
   122  	) (ineligibleTables,
   123  		eligibleTables []model.TableName, err error,
   124  	)
   125  }
   126  
   127  // APIV2HelpersImpl is an implementation of AVIV2Helpers interface
   128  type APIV2HelpersImpl struct{}
   129  
   130  // verifyCreateChangefeedConfig verifies ChangefeedConfig and
   131  // returns a changefeedInfo for create a changefeed.
   132  func (APIV2HelpersImpl) verifyCreateChangefeedConfig(
   133  	ctx context.Context,
   134  	cfg *ChangefeedConfig,
   135  	pdClient pd.Client,
   136  	ctrl controller.Controller,
   137  	ensureGCServiceID string,
   138  	kvStorage tidbkv.Storage,
   139  ) (*model.ChangeFeedInfo, error) {
   140  	// verify sinkURI
   141  	if cfg.SinkURI == "" {
   142  		return nil, cerror.ErrSinkURIInvalid.GenWithStackByArgs(
   143  			"sink_uri is empty, cannot create a changefeed without sink_uri")
   144  	}
   145  
   146  	// verify changefeedID
   147  	if cfg.ID == "" {
   148  		cfg.ID = uuid.New().String()
   149  	}
   150  	if err := model.ValidateChangefeedID(cfg.ID); err != nil {
   151  		return nil, cerror.ErrAPIInvalidParam.GenWithStack(
   152  			"invalid changefeed_id: %s", cfg.ID)
   153  	}
   154  	if cfg.Namespace == "" {
   155  		cfg.Namespace = model.DefaultNamespace
   156  	}
   157  
   158  	if err := model.ValidateNamespace(cfg.Namespace); err != nil {
   159  		return nil, cerror.ErrAPIInvalidParam.GenWithStack(
   160  			"invalid namespace: %s", cfg.Namespace)
   161  	}
   162  
   163  	exists, err := ctrl.IsChangefeedExists(ctx,
   164  		model.ChangeFeedID{Namespace: cfg.Namespace, ID: cfg.ID})
   165  	if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) {
   166  		return nil, err
   167  	}
   168  	if exists {
   169  		return nil, cerror.ErrChangeFeedAlreadyExists.GenWithStackByArgs(cfg.ID)
   170  	}
   171  
   172  	ts, logical, err := pdClient.GetTS(ctx)
   173  	if err != nil {
   174  		return nil, cerror.ErrPDEtcdAPIError.GenWithStackByArgs("fail to get ts from pd client")
   175  	}
   176  	currentTSO := oracle.ComposeTS(ts, logical)
   177  	// verify start ts
   178  	if cfg.StartTs == 0 {
   179  		cfg.StartTs = currentTSO
   180  	} else if cfg.StartTs > currentTSO {
   181  		return nil, cerror.ErrAPIInvalidParam.GenWithStack(
   182  			"invalid start-ts %v, larger than current tso %v", cfg.StartTs, currentTSO)
   183  	}
   184  	// Ensure the start ts is valid in the next 3600 seconds, aka 1 hour
   185  	const ensureTTL = 60 * 60
   186  	if err = gc.EnsureChangefeedStartTsSafety(
   187  		ctx,
   188  		pdClient,
   189  		ensureGCServiceID,
   190  		model.ChangeFeedID{Namespace: cfg.Namespace, ID: cfg.ID},
   191  		ensureTTL, cfg.StartTs); err != nil {
   192  		if !cerror.ErrStartTsBeforeGC.Equal(err) {
   193  			return nil, cerror.ErrPDEtcdAPIError.Wrap(err)
   194  		}
   195  		return nil, err
   196  	}
   197  
   198  	// verify target ts
   199  	if cfg.TargetTs > 0 && cfg.TargetTs <= cfg.StartTs {
   200  		return nil, cerror.ErrTargetTsBeforeStartTs.GenWithStackByArgs(
   201  			cfg.TargetTs, cfg.StartTs)
   202  	}
   203  
   204  	// fill replicaConfig
   205  	replicaCfg := cfg.ReplicaConfig.ToInternalReplicaConfig()
   206  	// verify replicaConfig
   207  	sinkURIParsed, err := url.Parse(cfg.SinkURI)
   208  	if err != nil {
   209  		return nil, cerror.WrapError(cerror.ErrSinkURIInvalid, err)
   210  	}
   211  	err = replicaCfg.ValidateAndAdjust(sinkURIParsed)
   212  	if err != nil {
   213  		return nil, err
   214  	}
   215  
   216  	f, err := filter.NewFilter(replicaCfg, "")
   217  	if err != nil {
   218  		return nil, errors.Cause(err)
   219  	}
   220  	tableInfos, ineligibleTables, _, err := entry.VerifyTables(f, kvStorage, cfg.StartTs)
   221  	if err != nil {
   222  		return nil, errors.Cause(err)
   223  	}
   224  	err = f.Verify(tableInfos)
   225  	if err != nil {
   226  		return nil, errors.Cause(err)
   227  	}
   228  	if !replicaCfg.ForceReplicate && !cfg.ReplicaConfig.IgnoreIneligibleTable {
   229  		if err != nil {
   230  			return nil, err
   231  		}
   232  		if len(ineligibleTables) != 0 {
   233  			return nil, cerror.ErrTableIneligible.GenWithStackByArgs(ineligibleTables)
   234  		}
   235  	}
   236  
   237  	// verify sink
   238  	if err = validator.Validate(ctx,
   239  		model.ChangeFeedID{Namespace: cfg.Namespace, ID: cfg.ID},
   240  		cfg.SinkURI, replicaCfg, nil,
   241  	); err != nil {
   242  		return nil, err
   243  	}
   244  
   245  	return &model.ChangeFeedInfo{
   246  		UpstreamID:     pdClient.GetClusterID(ctx),
   247  		Namespace:      cfg.Namespace,
   248  		ID:             cfg.ID,
   249  		SinkURI:        cfg.SinkURI,
   250  		CreateTime:     time.Now(),
   251  		StartTs:        cfg.StartTs,
   252  		TargetTs:       cfg.TargetTs,
   253  		Config:         replicaCfg,
   254  		State:          model.StateNormal,
   255  		CreatorVersion: version.ReleaseVersion,
   256  		Epoch:          owner.GenerateChangefeedEpoch(ctx, pdClient),
   257  	}, nil
   258  }
   259  
   260  // verifyUpstream verifies the upstream config before updating a changefeed
   261  func (h APIV2HelpersImpl) verifyUpstream(ctx context.Context,
   262  	changefeedConfig *ChangefeedConfig,
   263  	cfInfo *model.ChangeFeedInfo,
   264  ) error {
   265  	if len(changefeedConfig.PDAddrs) == 0 {
   266  		return nil
   267  	}
   268  	// check if the upstream cluster id changed
   269  	timeoutCtx, cancel := context.WithTimeout(ctx, 30*time.Second)
   270  	defer cancel()
   271  	pdClient, err := h.getPDClient(timeoutCtx, changefeedConfig.PDAddrs, &security.Credential{
   272  		CAPath:        changefeedConfig.CAPath,
   273  		CertPath:      changefeedConfig.CertPath,
   274  		KeyPath:       changefeedConfig.KeyPath,
   275  		CertAllowedCN: changefeedConfig.CertAllowedCN,
   276  	})
   277  	if err != nil {
   278  		return err
   279  	}
   280  	defer pdClient.Close()
   281  	if pdClient.GetClusterID(ctx) != cfInfo.UpstreamID {
   282  		return cerror.ErrUpstreamMissMatch.
   283  			GenWithStackByArgs(cfInfo.UpstreamID, pdClient.GetClusterID(ctx))
   284  	}
   285  
   286  	return nil
   287  }
   288  
   289  // verifyUpdateChangefeedConfig verifies config to update
   290  // a changefeed and returns a changefeedInfo
   291  func (APIV2HelpersImpl) verifyUpdateChangefeedConfig(
   292  	ctx context.Context,
   293  	cfg *ChangefeedConfig,
   294  	oldInfo *model.ChangeFeedInfo,
   295  	oldUpInfo *model.UpstreamInfo,
   296  	kvStorage tidbkv.Storage,
   297  	checkpointTs uint64,
   298  ) (*model.ChangeFeedInfo, *model.UpstreamInfo, error) {
   299  	// update changefeed info
   300  	newInfo, err := oldInfo.Clone()
   301  	if err != nil {
   302  		return nil, nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByArgs(err.Error())
   303  	}
   304  
   305  	var configUpdated, sinkURIUpdated bool
   306  	if cfg.TargetTs != 0 {
   307  		if cfg.TargetTs <= newInfo.StartTs {
   308  			return nil, nil, cerror.ErrChangefeedUpdateRefused.GenWithStack(
   309  				"can not update target_ts:%d less than start_ts:%d",
   310  				cfg.TargetTs, newInfo.StartTs)
   311  		}
   312  		newInfo.TargetTs = cfg.TargetTs
   313  	}
   314  	if cfg.ReplicaConfig != nil {
   315  		configUpdated = true
   316  		newInfo.Config = cfg.ReplicaConfig.ToInternalReplicaConfig()
   317  	}
   318  	if cfg.SinkURI != "" {
   319  		sinkURIUpdated = true
   320  		newInfo.SinkURI = cfg.SinkURI
   321  	}
   322  
   323  	// verify changefeed info
   324  	f, err := filter.NewFilter(newInfo.Config, "")
   325  	if err != nil {
   326  		return nil, nil, cerror.ErrChangefeedUpdateRefused.
   327  			GenWithStackByArgs(errors.Cause(err).Error())
   328  	}
   329  	tableInfos, _, _, err := entry.VerifyTables(f, kvStorage, checkpointTs)
   330  	if err != nil {
   331  		return nil, nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByCause(err)
   332  	}
   333  	err = f.Verify(tableInfos)
   334  	if err != nil {
   335  		return nil, nil, cerror.ErrChangefeedUpdateRefused.
   336  			GenWithStackByArgs(errors.Cause(err).Error())
   337  	}
   338  
   339  	if configUpdated || sinkURIUpdated {
   340  		log.Info("config or sink uri updated, check the compatibility",
   341  			zap.Bool("configUpdated", configUpdated),
   342  			zap.Bool("sinkURIUpdated", sinkURIUpdated))
   343  		// check sink config is compatible with sinkURI
   344  		newCfg := newInfo.Config.Sink
   345  		oldCfg := oldInfo.Config.Sink
   346  		err := newCfg.CheckCompatibilityWithSinkURI(oldCfg, newInfo.SinkURI)
   347  		if err != nil {
   348  			return nil, nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByCause(err)
   349  		}
   350  
   351  		// use the sinkURI to validate and adjust the new config
   352  		sinkURI := oldInfo.SinkURI
   353  		if sinkURIUpdated {
   354  			sinkURI = newInfo.SinkURI
   355  		}
   356  		sinkURIParsed, err := url.Parse(sinkURI)
   357  		if err != nil {
   358  			return nil, nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByCause(err)
   359  		}
   360  		err = newInfo.Config.ValidateAndAdjust(sinkURIParsed)
   361  		if err != nil {
   362  			return nil, nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByCause(err)
   363  		}
   364  
   365  		if err := validator.Validate(ctx,
   366  			model.ChangeFeedID{Namespace: cfg.Namespace, ID: cfg.ID},
   367  			newInfo.SinkURI, newInfo.Config, nil); err != nil {
   368  			return nil, nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByCause(err)
   369  		}
   370  	}
   371  
   372  	// update and verify up info
   373  	newUpInfo, err := oldUpInfo.Clone()
   374  	if err != nil {
   375  		return nil, nil, cerror.ErrChangefeedUpdateRefused.GenWithStackByArgs(err.Error())
   376  	}
   377  	if cfg.PDAddrs != nil {
   378  		newUpInfo.PDEndpoints = strings.Join(cfg.PDAddrs, ",")
   379  	}
   380  	if cfg.CAPath != "" {
   381  		newUpInfo.CAPath = cfg.CAPath
   382  	}
   383  	if cfg.CertPath != "" {
   384  		newUpInfo.CertPath = cfg.CertPath
   385  	}
   386  	if cfg.KeyPath != "" {
   387  		newUpInfo.KeyPath = cfg.KeyPath
   388  	}
   389  	if cfg.CertAllowedCN != nil {
   390  		newUpInfo.CertAllowedCN = cfg.CertAllowedCN
   391  	}
   392  	changefeedInfoChanged := diff.Changed(oldInfo, newInfo)
   393  	upstreamInfoChanged := diff.Changed(oldUpInfo, newUpInfo)
   394  	if !changefeedInfoChanged && !upstreamInfoChanged {
   395  		return nil, nil, cerror.ErrChangefeedUpdateRefused.
   396  			GenWithStackByArgs("changefeed config is the same with the old one, do nothing")
   397  	}
   398  	return newInfo, newUpInfo, nil
   399  }
   400  
   401  // verifyResumeChangefeedConfig verifies the changefeed config before resuming a changefeed
   402  // overrideCheckpointTs is the checkpointTs of the changefeed that specified by the user.
   403  // or it is the checkpointTs of the changefeed before it is paused.
   404  // we need to check weather the resuming changefeed is gc safe or not.
   405  func (APIV2HelpersImpl) verifyResumeChangefeedConfig(ctx context.Context,
   406  	pdClient pd.Client,
   407  	gcServiceID string,
   408  	changefeedID model.ChangeFeedID,
   409  	overrideCheckpointTs uint64,
   410  ) error {
   411  	if overrideCheckpointTs == 0 {
   412  		return nil
   413  	}
   414  
   415  	ts, logical, err := pdClient.GetTS(ctx)
   416  	if err != nil {
   417  		return cerror.ErrPDEtcdAPIError.GenWithStackByArgs("fail to get ts from pd client")
   418  	}
   419  	currentTSO := oracle.ComposeTS(ts, logical)
   420  	if overrideCheckpointTs > currentTSO {
   421  		return cerror.ErrAPIInvalidParam.GenWithStack(
   422  			"invalid checkpoint-ts %v, larger than current tso %v", overrideCheckpointTs, currentTSO)
   423  	}
   424  
   425  	// 1h is enough for resuming a changefeed.
   426  	gcTTL := int64(60 * 60)
   427  	err = gc.EnsureChangefeedStartTsSafety(
   428  		ctx,
   429  		pdClient,
   430  		gcServiceID,
   431  		changefeedID,
   432  		gcTTL, overrideCheckpointTs)
   433  	if err != nil {
   434  		if !cerror.ErrStartTsBeforeGC.Equal(err) {
   435  			return cerror.ErrPDEtcdAPIError.Wrap(err)
   436  		}
   437  		return err
   438  	}
   439  
   440  	return nil
   441  }
   442  
   443  // getPDClient returns a PDClient given the PD cluster addresses and a credential
   444  func (APIV2HelpersImpl) getPDClient(ctx context.Context,
   445  	pdAddrs []string,
   446  	credential *security.Credential,
   447  ) (pd.Client, error) {
   448  	grpcTLSOption, err := credential.ToGRPCDialOption()
   449  	if err != nil {
   450  		return nil, errors.Trace(err)
   451  	}
   452  
   453  	pdClient, err := pd.NewClientWithContext(
   454  		ctx, pdAddrs, credential.PDSecurityOption(),
   455  		pd.WithGRPCDialOptions(
   456  			grpcTLSOption,
   457  			grpc.WithBlock(),
   458  			grpc.WithConnectParams(grpc.ConnectParams{
   459  				Backoff: backoff.Config{
   460  					BaseDelay:  time.Second,
   461  					Multiplier: 1.1,
   462  					Jitter:     0.1,
   463  					MaxDelay:   3 * time.Second,
   464  				},
   465  				MinConnectTimeout: 3 * time.Second,
   466  			}),
   467  		),
   468  		pd.WithForwardingOption(config.EnablePDForwarding))
   469  	if err != nil {
   470  		return nil, cerror.WrapError(cerror.ErrAPIGetPDClientFailed, errors.Trace(err))
   471  	}
   472  	return pdClient, nil
   473  }
   474  
   475  func (h APIV2HelpersImpl) getEtcdClient(
   476  	pdAddrs []string, tlsCfg *tls.Config,
   477  ) (*clientv3.Client, error) {
   478  	conf := config.GetGlobalServerConfig()
   479  	grpcTLSOption, err := conf.Security.ToGRPCDialOption()
   480  	if err != nil {
   481  		return nil, err
   482  	}
   483  	logConfig := &logutil.DefaultZapLoggerConfig
   484  	logConfig.Level = zap.NewAtomicLevelAt(zapcore.ErrorLevel)
   485  	return clientv3.New(
   486  		clientv3.Config{
   487  			Endpoints:   pdAddrs,
   488  			TLS:         tlsCfg,
   489  			LogConfig:   logConfig,
   490  			DialTimeout: 5 * time.Second,
   491  			DialOptions: []grpc.DialOption{
   492  				grpcTLSOption,
   493  				grpc.WithBlock(),
   494  				grpc.WithConnectParams(
   495  					grpc.ConnectParams{
   496  						Backoff: backoff.Config{
   497  							BaseDelay:  time.Second,
   498  							Multiplier: 1.1,
   499  							Jitter:     0.1,
   500  							MaxDelay:   3 * time.Second,
   501  						},
   502  						MinConnectTimeout: 3 * time.Second,
   503  					},
   504  				),
   505  			},
   506  		},
   507  	)
   508  }
   509  
   510  // getTiStore wrap the kv.createTiStore method to increase testability
   511  func (h APIV2HelpersImpl) createTiStore(pdAddrs []string,
   512  	credential *security.Credential,
   513  ) (tidbkv.Storage, error) {
   514  	return kv.CreateTiStore(strings.Join(pdAddrs, ","), credential)
   515  }
   516  
   517  func (h APIV2HelpersImpl) getVerifiedTables(
   518  	replicaConfig *config.ReplicaConfig,
   519  	storage tidbkv.Storage, startTs uint64,
   520  	scheme string, topic string, protocol config.Protocol,
   521  ) ([]model.TableName, []model.TableName, error) {
   522  	f, err := filter.NewFilter(replicaConfig, "")
   523  	if err != nil {
   524  		return nil, nil, err
   525  	}
   526  	tableInfos, ineligibleTables, eligibleTables, err := entry.
   527  		VerifyTables(f, storage, startTs)
   528  	if err != nil {
   529  		return nil, nil, err
   530  	}
   531  
   532  	if !sink.IsMQScheme(scheme) {
   533  		return ineligibleTables, eligibleTables, nil
   534  	}
   535  
   536  	eventRouter, err := dispatcher.NewEventRouter(replicaConfig, protocol, topic, scheme)
   537  	if err != nil {
   538  		return nil, nil, err
   539  	}
   540  	err = eventRouter.VerifyTables(tableInfos)
   541  	if err != nil {
   542  		return nil, nil, err
   543  	}
   544  
   545  	selectors, err := columnselector.New(replicaConfig)
   546  	if err != nil {
   547  		return nil, nil, err
   548  	}
   549  	err = selectors.VerifyTables(tableInfos, eventRouter)
   550  	if err != nil {
   551  		return nil, nil, err
   552  	}
   553  
   554  	return ineligibleTables, eligibleTables, nil
   555  }