github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/filter/expr_filter.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 filter
    15  
    16  import (
    17  	"strings"
    18  	"sync"
    19  
    20  	"github.com/pingcap/errors"
    21  	"github.com/pingcap/log"
    22  	"github.com/pingcap/tidb/pkg/expression"
    23  	"github.com/pingcap/tidb/pkg/parser"
    24  	"github.com/pingcap/tidb/pkg/sessionctx"
    25  	"github.com/pingcap/tidb/pkg/types"
    26  	"github.com/pingcap/tidb/pkg/util/chunk"
    27  	"github.com/pingcap/tidb/pkg/util/dbterror/plannererrors"
    28  	tfilter "github.com/pingcap/tidb/pkg/util/table-filter"
    29  	"github.com/pingcap/tiflow/cdc/model"
    30  	"github.com/pingcap/tiflow/dm/pkg/utils"
    31  	"github.com/pingcap/tiflow/pkg/config"
    32  	cerror "github.com/pingcap/tiflow/pkg/errors"
    33  	"go.uber.org/zap"
    34  )
    35  
    36  // dmlExprFilterRule only be used by dmlExprFilter.
    37  // This struct is mostly a duplicate of `ExprFilterGroup` in dm/pkg/syncer,
    38  // but have slightly changed to fit the usage of cdc.
    39  type dmlExprFilterRule struct {
    40  	mu sync.Mutex
    41  	// Cache tableInfos to check if the table was changed.
    42  	tables map[string]*model.TableInfo
    43  
    44  	insertExprs    map[string]expression.Expression // tableName -> expr
    45  	updateOldExprs map[string]expression.Expression // tableName -> expr
    46  	updateNewExprs map[string]expression.Expression // tableName -> expr
    47  	deleteExprs    map[string]expression.Expression // tableName -> expr
    48  
    49  	tableMatcher tfilter.Filter
    50  	// All tables in this rule share the same config.
    51  	config *config.EventFilterRule
    52  
    53  	sessCtx sessionctx.Context
    54  }
    55  
    56  func newExprFilterRule(
    57  	sessCtx sessionctx.Context,
    58  	cfg *config.EventFilterRule,
    59  ) (*dmlExprFilterRule, error) {
    60  	tf, err := tfilter.Parse(cfg.Matcher)
    61  	if err != nil {
    62  		return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, cfg.Matcher)
    63  	}
    64  
    65  	ret := &dmlExprFilterRule{
    66  		tables:         make(map[string]*model.TableInfo),
    67  		insertExprs:    make(map[string]expression.Expression),
    68  		updateOldExprs: make(map[string]expression.Expression),
    69  		updateNewExprs: make(map[string]expression.Expression),
    70  		deleteExprs:    make(map[string]expression.Expression),
    71  		config:         cfg,
    72  		tableMatcher:   tf,
    73  		sessCtx:        sessCtx,
    74  	}
    75  	return ret, nil
    76  }
    77  
    78  // verifyAndInitRule will verify and init the rule.
    79  // It should only be called in dmlExprFilter's verify method.
    80  // We ask users to set these expr only in default sql mode,
    81  // so we just need to  verify each expr in default sql mode
    82  func (r *dmlExprFilterRule) verify(tableInfos []*model.TableInfo) error {
    83  	// verify expression filter rule syntax.
    84  	p := parser.New()
    85  	_, _, err := p.ParseSQL(completeExpression(r.config.IgnoreInsertValueExpr))
    86  	if err != nil {
    87  		log.Error("failed to parse expression", zap.Error(err))
    88  		return cerror.ErrExpressionParseFailed.
    89  			FastGenByArgs(r.config.IgnoreInsertValueExpr)
    90  	}
    91  	_, _, err = p.ParseSQL(completeExpression(r.config.IgnoreUpdateNewValueExpr))
    92  	if err != nil {
    93  		log.Error("failed to parse expression", zap.Error(err))
    94  		return cerror.ErrExpressionParseFailed.
    95  			FastGenByArgs(r.config.IgnoreUpdateNewValueExpr)
    96  	}
    97  	_, _, err = p.ParseSQL(completeExpression(r.config.IgnoreUpdateOldValueExpr))
    98  	if err != nil {
    99  		log.Error("failed to parse expression", zap.Error(err))
   100  		return cerror.ErrExpressionParseFailed.
   101  			FastGenByArgs(r.config.IgnoreUpdateOldValueExpr)
   102  	}
   103  	_, _, err = p.ParseSQL(completeExpression(r.config.IgnoreDeleteValueExpr))
   104  	if err != nil {
   105  		log.Error("failed to parse expression", zap.Error(err))
   106  		return cerror.ErrExpressionParseFailed.
   107  			FastGenByArgs(r.config.IgnoreDeleteValueExpr)
   108  	}
   109  	// verify expression filter rule.
   110  	for _, ti := range tableInfos {
   111  		tableName := ti.TableName.String()
   112  		if !r.tableMatcher.MatchTable(ti.TableName.Schema, ti.TableName.Table) {
   113  			continue
   114  		}
   115  		if r.config.IgnoreInsertValueExpr != "" {
   116  			e, err := r.getSimpleExprOfTable(r.config.IgnoreInsertValueExpr, ti)
   117  			if err != nil {
   118  				return err
   119  			}
   120  			r.insertExprs[tableName] = e
   121  		}
   122  		if r.config.IgnoreUpdateOldValueExpr != "" {
   123  			e, err := r.getSimpleExprOfTable(r.config.IgnoreUpdateOldValueExpr, ti)
   124  			if err != nil {
   125  				return err
   126  			}
   127  			r.updateOldExprs[tableName] = e
   128  		}
   129  		if r.config.IgnoreUpdateNewValueExpr != "" {
   130  			e, err := r.getSimpleExprOfTable(r.config.IgnoreUpdateNewValueExpr, ti)
   131  			if err != nil {
   132  				return err
   133  			}
   134  			r.updateNewExprs[tableName] = e
   135  		}
   136  		if r.config.IgnoreDeleteValueExpr != "" {
   137  			e, err := r.getSimpleExprOfTable(r.config.IgnoreDeleteValueExpr, ti)
   138  			if err != nil {
   139  				return err
   140  			}
   141  			r.deleteExprs[tableName] = e
   142  		}
   143  	}
   144  	return nil
   145  }
   146  
   147  // The caller must hold r.mu.Lock() before calling this function.
   148  func (r *dmlExprFilterRule) resetExpr(tableName string) {
   149  	delete(r.insertExprs, tableName)
   150  	delete(r.updateOldExprs, tableName)
   151  	delete(r.updateNewExprs, tableName)
   152  	delete(r.deleteExprs, tableName)
   153  }
   154  
   155  // getInsertExprs returns the expression filter to filter INSERT events.
   156  // This function will lazy calculate expressions if not initialized.
   157  func (r *dmlExprFilterRule) getInsertExpr(ti *model.TableInfo) (
   158  	expression.Expression, error,
   159  ) {
   160  	tableName := ti.TableName.String()
   161  	if r.insertExprs[tableName] != nil {
   162  		return r.insertExprs[tableName], nil
   163  	}
   164  	if r.config.IgnoreInsertValueExpr != "" {
   165  		expr, err := r.getSimpleExprOfTable(r.config.IgnoreInsertValueExpr, ti)
   166  		if err != nil {
   167  			return nil, err
   168  		}
   169  		r.insertExprs[tableName] = expr
   170  	}
   171  	return r.insertExprs[tableName], nil
   172  }
   173  
   174  func (r *dmlExprFilterRule) getUpdateOldExpr(ti *model.TableInfo) (
   175  	expression.Expression, error,
   176  ) {
   177  	tableName := ti.TableName.String()
   178  	if r.updateOldExprs[tableName] != nil {
   179  		return r.updateOldExprs[tableName], nil
   180  	}
   181  
   182  	if r.config.IgnoreUpdateOldValueExpr != "" {
   183  		expr, err := r.getSimpleExprOfTable(r.config.IgnoreUpdateOldValueExpr, ti)
   184  		if err != nil {
   185  			return nil, err
   186  		}
   187  		r.updateOldExprs[tableName] = expr
   188  	}
   189  	return r.updateOldExprs[tableName], nil
   190  }
   191  
   192  func (r *dmlExprFilterRule) getUpdateNewExpr(ti *model.TableInfo) (
   193  	expression.Expression, error,
   194  ) {
   195  	tableName := ti.TableName.String()
   196  	if r.updateNewExprs[tableName] != nil {
   197  		return r.updateNewExprs[tableName], nil
   198  	}
   199  
   200  	if r.config.IgnoreUpdateNewValueExpr != "" {
   201  		expr, err := r.getSimpleExprOfTable(r.config.IgnoreUpdateNewValueExpr, ti)
   202  		if err != nil {
   203  			return nil, err
   204  		}
   205  		r.updateNewExprs[tableName] = expr
   206  	}
   207  	return r.updateNewExprs[tableName], nil
   208  }
   209  
   210  func (r *dmlExprFilterRule) getDeleteExpr(ti *model.TableInfo) (
   211  	expression.Expression, error,
   212  ) {
   213  	tableName := ti.TableName.String()
   214  	if r.deleteExprs[tableName] != nil {
   215  		return r.deleteExprs[tableName], nil
   216  	}
   217  
   218  	if r.config.IgnoreDeleteValueExpr != "" {
   219  		expr, err := r.getSimpleExprOfTable(r.config.IgnoreDeleteValueExpr, ti)
   220  		if err != nil {
   221  			return nil, err
   222  		}
   223  		r.deleteExprs[tableName] = expr
   224  	}
   225  	return r.deleteExprs[tableName], nil
   226  }
   227  
   228  func (r *dmlExprFilterRule) getSimpleExprOfTable(
   229  	expr string,
   230  	ti *model.TableInfo,
   231  ) (expression.Expression, error) {
   232  	e, err := expression.ParseSimpleExprWithTableInfo(r.sessCtx.GetExprCtx(), expr, ti.TableInfo)
   233  	if err != nil {
   234  		// If an expression contains an unknown column,
   235  		// we return an error and stop the changefeed.
   236  		if plannererrors.ErrUnknownColumn.Equal(err) {
   237  			log.Error("meet unknown column when generating expression",
   238  				zap.String("expression", expr),
   239  				zap.Error(err))
   240  			return nil, cerror.ErrExpressionColumnNotFound.
   241  				FastGenByArgs(getColumnFromError(err), ti.TableName.String(), expr)
   242  		}
   243  		log.Error("failed to parse expression", zap.Error(err))
   244  		return nil, cerror.ErrExpressionParseFailed.FastGenByArgs(err, expr)
   245  	}
   246  	return e, nil
   247  }
   248  
   249  func (r *dmlExprFilterRule) shouldSkipDML(
   250  	row *model.RowChangedEvent,
   251  	rawRow model.RowChangedDatums,
   252  	ti *model.TableInfo,
   253  ) (bool, error) {
   254  	tableName := ti.TableName.String()
   255  
   256  	r.mu.Lock()
   257  	defer r.mu.Unlock()
   258  
   259  	if oldTi, ok := r.tables[tableName]; ok {
   260  		// If one table's tableInfo was updated, we need to reset this rule
   261  		// and update the tableInfo in the cache.
   262  		if ti.Version != oldTi.Version {
   263  			r.tables[tableName] = ti.Clone()
   264  			r.resetExpr(ti.TableName.String())
   265  		}
   266  	} else {
   267  		r.tables[tableName] = ti.Clone()
   268  	}
   269  
   270  	switch {
   271  	case row.IsInsert():
   272  		exprs, err := r.getInsertExpr(ti)
   273  		if err != nil {
   274  			return false, err
   275  		}
   276  		return r.skipDMLByExpression(
   277  			rawRow.RowDatums,
   278  			exprs,
   279  		)
   280  	case row.IsUpdate():
   281  		oldExprs, err := r.getUpdateOldExpr(ti)
   282  		if err != nil {
   283  			return false, err
   284  		}
   285  		newExprs, err := r.getUpdateNewExpr(ti)
   286  		if err != nil {
   287  			return false, err
   288  		}
   289  		ignoreOld, err := r.skipDMLByExpression(
   290  			rawRow.PreRowDatums,
   291  			oldExprs,
   292  		)
   293  		if err != nil {
   294  			return false, err
   295  		}
   296  		ignoreNew, err := r.skipDMLByExpression(
   297  			rawRow.RowDatums,
   298  			newExprs,
   299  		)
   300  		if err != nil {
   301  			return false, err
   302  		}
   303  		return ignoreOld || ignoreNew, nil
   304  	case row.IsDelete():
   305  		exprs, err := r.getDeleteExpr(ti)
   306  		if err != nil {
   307  			return false, err
   308  		}
   309  		return r.skipDMLByExpression(
   310  			rawRow.PreRowDatums,
   311  			exprs,
   312  		)
   313  	default:
   314  		log.Warn("unknown row changed event type")
   315  		return false, nil
   316  	}
   317  }
   318  
   319  func (r *dmlExprFilterRule) skipDMLByExpression(
   320  	rowData []types.Datum,
   321  	expr expression.Expression,
   322  ) (bool, error) {
   323  	if len(rowData) == 0 || expr == nil {
   324  		return false, nil
   325  	}
   326  
   327  	row := chunk.MutRowFromDatums(rowData).ToRow()
   328  
   329  	d, err := expr.Eval(r.sessCtx.GetExprCtx().GetEvalCtx(), row)
   330  	if err != nil {
   331  		log.Error("failed to eval expression", zap.Error(err))
   332  		return false, errors.Trace(err)
   333  	}
   334  	if d.GetInt64() == 1 {
   335  		return true, nil
   336  	}
   337  	return false, nil
   338  }
   339  
   340  func getColumnFromError(err error) string {
   341  	if !plannererrors.ErrUnknownColumn.Equal(err) {
   342  		return err.Error()
   343  	}
   344  	column := strings.TrimSpace(strings.TrimPrefix(err.Error(),
   345  		"[planner:1054]Unknown column '"))
   346  	column = strings.TrimSuffix(column, "' in 'expression'")
   347  	return column
   348  }
   349  
   350  // dmlExprFilter is a filter that filters DML events by SQL expression.
   351  type dmlExprFilter struct {
   352  	rules []*dmlExprFilterRule
   353  }
   354  
   355  func newExprFilter(
   356  	timezone string,
   357  	cfg *config.FilterConfig,
   358  ) (*dmlExprFilter, error) {
   359  	res := &dmlExprFilter{}
   360  	sessCtx := utils.NewSessionCtx(map[string]string{
   361  		"time_zone": timezone,
   362  	})
   363  	for _, rule := range cfg.EventFilters {
   364  		err := res.addRule(sessCtx, rule)
   365  		if err != nil {
   366  			return nil, err
   367  		}
   368  	}
   369  	return res, nil
   370  }
   371  
   372  func (f *dmlExprFilter) addRule(
   373  	sessCtx sessionctx.Context,
   374  	cfg *config.EventFilterRule,
   375  ) error {
   376  	rule, err := newExprFilterRule(sessCtx, cfg)
   377  	if err != nil {
   378  		return errors.Trace(err)
   379  	}
   380  	f.rules = append(f.rules, rule)
   381  	return nil
   382  }
   383  
   384  // verify checks if all rules in this filter is valid.
   385  func (f *dmlExprFilter) verify(tableInfos []*model.TableInfo) error {
   386  	for _, rule := range f.rules {
   387  		err := rule.verify(tableInfos)
   388  		if err != nil {
   389  			log.Error("failed to verify expression filter rule", zap.Error(err))
   390  			return errors.Trace(err)
   391  		}
   392  	}
   393  	return nil
   394  }
   395  
   396  func (f *dmlExprFilter) getRules(schema, table string) []*dmlExprFilterRule {
   397  	res := make([]*dmlExprFilterRule, 0)
   398  	for _, rule := range f.rules {
   399  		if rule.tableMatcher.MatchTable(schema, table) {
   400  			res = append(res, rule)
   401  		}
   402  	}
   403  	return res
   404  }
   405  
   406  // shouldSkipDML skips dml event by sql expression.
   407  func (f *dmlExprFilter) shouldSkipDML(
   408  	row *model.RowChangedEvent,
   409  	rawRow model.RowChangedDatums,
   410  	ti *model.TableInfo,
   411  ) (bool, error) {
   412  	if len(f.rules) == 0 {
   413  		return false, nil
   414  	}
   415  	// for defense purpose, normally the row and ti should not be nil.
   416  	if ti == nil || row == nil || rawRow.IsEmpty() {
   417  		return false, nil
   418  	}
   419  	rules := f.getRules(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName())
   420  	for _, rule := range rules {
   421  		ignore, err := rule.shouldSkipDML(row, rawRow, ti)
   422  		if err != nil {
   423  			if cerror.ShouldFailChangefeed(err) {
   424  				return false, err
   425  			}
   426  			return false, cerror.WrapError(cerror.ErrFailedToFilterDML, err, row)
   427  		}
   428  		if ignore {
   429  			return true, nil
   430  		}
   431  	}
   432  	return false, nil
   433  }